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

[01/45] ignite git commit: ignite-1353: predefined ID is now properly determined during serialization/deserialization

Repository: ignite
Updated Branches:
  refs/heads/ignite-843 e5eede56b -> ce1559bc8


ignite-1353: predefined ID is now properly determined during serialization/deserialization


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

Branch: refs/heads/ignite-843
Commit: 4d3c1f07b850a6519908c6d4383ec81310bbab40
Parents: 58a665a
Author: Denis Magda <dm...@gridgain.com>
Authored: Fri Sep 4 13:34:58 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Fri Sep 4 13:34:58 2015 +0300

----------------------------------------------------------------------
 .../portable/PortableClassDescriptor.java       |  79 ++++++++-----
 .../internal/portable/PortableContext.java      | 113 +++++++++----------
 .../internal/portable/PortableWriterExImpl.java |  92 ++++-----------
 .../GridPortableMarshallerSelfTest.java         |  68 +++++++----
 .../portable/GridPortableWildcardsSelfTest.java |  60 +++++-----
 5 files changed, 208 insertions(+), 204 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4d3c1f07/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
index 24ad5ce..a2b4b74 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
@@ -19,10 +19,13 @@ package org.apache.ignite.internal.portable;
 
 import java.io.Externalizable;
 import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.Field;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
 import java.math.BigDecimal;
 import java.sql.Timestamp;
 import java.util.ArrayList;
@@ -36,6 +39,8 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.CacheObjectImpl;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.marshaller.MarshallerExclusions;
+import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
 import org.apache.ignite.portable.PortableException;
 import org.apache.ignite.portable.PortableIdMapper;
 import org.apache.ignite.portable.PortableMarshalAware;
@@ -95,36 +100,10 @@ public class PortableClassDescriptor {
     private final boolean registered;
 
     /** */
-    private final boolean excluded;
+    private final boolean useOptMarshaller;
 
-    /**
-     * @param ctx Context.
-     * @param cls Class.
-     * @param userType User type flag.
-     * @param typeId Type ID.
-     * @param typeName Type name.
-     * @param idMapper ID mapper.
-     * @param serializer Serializer.
-     * @param useTs Use timestamp flag.
-     * @param metaDataEnabled Metadata enabled flag.
-     * @param keepDeserialized Keep deserialized flag.
-     * @throws PortableException In case of error.
-     */
-    PortableClassDescriptor(
-        PortableContext ctx,
-        Class<?> cls,
-        boolean userType,
-        int typeId,
-        String typeName,
-        @Nullable PortableIdMapper idMapper,
-        @Nullable PortableSerializer serializer,
-        boolean useTs,
-        boolean metaDataEnabled,
-        boolean keepDeserialized
-    ) throws PortableException {
-        this(ctx, cls, userType, typeId, typeName, idMapper, serializer, useTs, metaDataEnabled, keepDeserialized,
-            true);
-    }
+    /** */
+    private final boolean excluded;
 
     /**
      * @param ctx Context.
@@ -138,6 +117,7 @@ public class PortableClassDescriptor {
      * @param metaDataEnabled Metadata enabled flag.
      * @param keepDeserialized Keep deserialized flag.
      * @param registered Whether typeId has been successfully registered by MarshallerContext or not.
+     * @param predefined Whether the class is predefined or not.
      * @throws PortableException In case of error.
      */
     PortableClassDescriptor(
@@ -151,7 +131,8 @@ public class PortableClassDescriptor {
         boolean useTs,
         boolean metaDataEnabled,
         boolean keepDeserialized,
-        boolean registered
+        boolean registered,
+        boolean predefined
     ) throws PortableException {
         assert ctx != null;
         assert cls != null;
@@ -168,6 +149,8 @@ public class PortableClassDescriptor {
 
         excluded = MarshallerExclusions.isExcluded(cls);
 
+        useOptMarshaller = !predefined && initUseOptimizedMarshallerFlag();
+
         if (excluded)
             mode = Mode.EXCLUSION;
         else
@@ -313,11 +296,19 @@ public class PortableClassDescriptor {
     /**
      * @return Whether typeId has been successfully registered by MarshallerContext or not.
      */
-    public boolean isRegistered() {
+    public boolean registered() {
         return registered;
     }
 
     /**
+     * @return {@code true} if {@link OptimizedMarshaller} must be used instead of {@link PortableMarshaller}
+     * for object serialization and deserialization.
+     */
+    public boolean useOptimizedMarshaller() {
+        return useOptMarshaller;
+    }
+
+    /**
      * Checks whether the class values are explicitly excluded from marshalling.
      *
      * @return {@code true} if excluded, {@code false} otherwise.
@@ -720,6 +711,32 @@ public class PortableClassDescriptor {
     }
 
     /**
+     * Determines whether to use {@link OptimizedMarshaller} for serialization or
+     * not.
+     *
+     * @return {@code true} if to use, {@code false} otherwise.
+     */
+    private boolean initUseOptimizedMarshallerFlag() {
+       boolean use;
+
+        try {
+            Method writeObj = cls.getDeclaredMethod("writeObject", ObjectOutputStream.class);
+            Method readObj = cls.getDeclaredMethod("readObject", ObjectInputStream.class);
+
+            if (!Modifier.isStatic(writeObj.getModifiers()) && !Modifier.isStatic(readObj.getModifiers()) &&
+                writeObj.getReturnType() == void.class && readObj.getReturnType() == void.class)
+                use = true;
+            else
+                use = false;
+        }
+        catch (NoSuchMethodException e) {
+            use = false;
+        }
+
+        return use;
+    }
+
+    /**
      * @param cls Class.
      * @return Mode.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d3c1f07/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
index db7e41e..33a105b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
@@ -112,7 +112,7 @@ public class PortableContext implements Externalizable {
     private final Map<Integer, PortableClassDescriptor> predefinedTypes = new HashMap<>();
 
     /** */
-    private final Set<Class> predefinedClasses = new HashSet<>();
+    private final Map<String, Integer> predefinedTypeNames = new HashMap<>();
 
     /** */
     private final Map<Class<? extends Collection>, Byte> colTypes = new HashMap<>();
@@ -432,7 +432,7 @@ public class PortableContext implements Externalizable {
 
         PortableClassDescriptor desc = descByCls.get(cls);
 
-        if (desc == null || !desc.isRegistered())
+        if (desc == null || !desc.registered())
             desc = registerClassDescriptor(cls);
 
         return desc;
@@ -447,11 +447,22 @@ public class PortableContext implements Externalizable {
     public PortableClassDescriptor descriptorForTypeId(boolean userType, int typeId, ClassLoader ldr) {
         assert typeId != GridPortableMarshaller.UNREGISTERED_TYPE_ID;
 
-        PortableClassDescriptor desc = userType ? userTypes.get(typeId) : predefinedTypes.get(typeId);
+        //TODO: IGNITE-1358 (uncomment when fixed)
+        //PortableClassDescriptor desc = userType ? userTypes.get(typeId) : predefinedTypes.get(typeId);
+
+        // As a workaround for IGNITE-1358 we always check the predefined map before.
+        PortableClassDescriptor desc = predefinedTypes.get(typeId);
 
         if (desc != null)
             return desc;
 
+        if (userType) {
+            desc = userTypes.get(typeId);
+
+            if (desc != null)
+                return desc;
+        }
+
         Class cls;
 
         try {
@@ -496,7 +507,10 @@ public class PortableContext implements Externalizable {
                 null,
                 useTs,
                 metaDataEnabled,
-                keepDeserialized);
+                keepDeserialized,
+                true, /* registered */
+                false /* predefined */
+            );
 
             PortableClassDescriptor old = descByCls.putIfAbsent(cls, desc);
 
@@ -529,7 +543,8 @@ public class PortableContext implements Externalizable {
         try {
             registered = marshCtx.registerClass(typeId, cls);
 
-        } catch (IgniteCheckedException e) {
+        }
+        catch (IgniteCheckedException e) {
             throw new PortableException("Failed to register class.", e);
         }
 
@@ -543,7 +558,9 @@ public class PortableContext implements Externalizable {
             useTs,
             metaDataEnabled,
             keepDeserialized,
-            registered);
+            registered,
+            false /* predefined */
+        );
 
         // perform put() instead of putIfAbsent() because "registered" flag may have been changed.
         userTypes.put(typeId, desc);
@@ -584,43 +601,17 @@ public class PortableContext implements Externalizable {
      * @return Type ID.
      */
     public int typeId(String typeName) {
-        int id;
-
-        if (marshCtx.isSystemType(typeName))
-            id = typeName.hashCode();
-
-        else {
-            typeName = typeName(typeName);
-
-            id = idMapper(typeName).typeId(typeName);
-        }
-
-        return id;
-    }
-
-    /**
-     * @param cls Class.
-     * @return Type ID.
-     * @throws PortableException In case of error.
-     */
-    public Type typeId(Class cls) throws PortableException {
-        String clsName = cls.getName();
-
-        if (marshCtx.isSystemType(clsName))
-            return new Type(clsName.hashCode(), true);
-
-        if (predefinedClasses.contains(cls))
-            return new Type(DFLT_ID_MAPPER.typeId(typeName(clsName)), true);
+        String shortTypeName = typeName(typeName);
 
-        PortableClassDescriptor desc = descByCls.get(cls);
+        Integer id = predefinedTypeNames.get(shortTypeName);
 
-        boolean registered = desc != null && desc.isRegistered();
+        if (id != null)
+            return id;
 
-        if (!registered)
-            // forces to register the class and fill up all required data structures
-            desc = registerUserClassDescriptor(cls);
+        if (marshCtx.isSystemType(typeName))
+            return typeName.hashCode();
 
-        return new Type(desc.typeId(), desc.isRegistered());
+        return idMapper(shortTypeName).typeId(shortTypeName);
     }
 
     /**
@@ -692,22 +683,26 @@ public class PortableContext implements Externalizable {
      * @return GridPortableClassDescriptor.
      */
     public PortableClassDescriptor registerPredefinedType(Class<?> cls, int id) {
+        String typeName = typeName(cls.getName());
+
         PortableClassDescriptor desc = new PortableClassDescriptor(
             this,
             cls,
             false,
             id,
-            typeName(cls.getName()),
+            typeName,
             DFLT_ID_MAPPER,
             null,
             false,
             false,
-            false
+            false,
+            true, /* registered */
+            true /* predefined */
         );
 
-        predefinedClasses.add(cls);
-
+        predefinedTypeNames.put(typeName, id);
         predefinedTypes.put(id, desc);
+
         descByCls.put(cls, desc);
 
         return desc;
@@ -745,6 +740,10 @@ public class PortableContext implements Externalizable {
 
         int id = idMapper.typeId(clsName);
 
+        //Workaround for IGNITE-1358
+        if (predefinedTypes.get(id) != null)
+            throw new PortableException("Duplicate type ID [clsName=" + clsName + ", id=" + id + ']');
+
         if (mappers.put(id, idMapper) != null)
             throw new PortableException("Duplicate type ID [clsName=" + clsName + ", id=" + id + ']');
 
@@ -770,7 +769,10 @@ public class PortableContext implements Externalizable {
                 serializer,
                 useTs,
                 metaDataEnabled,
-                keepDeserialized);
+                keepDeserialized,
+                true, /* registered */
+                false /* predefined */
+            );
 
             fieldsMeta = desc.fieldsMeta();
 
@@ -863,16 +865,6 @@ public class PortableContext implements Externalizable {
     }
 
     /**
-     * Returns whether {@code cls} is predefined in the context or not.
-     *
-     * @param cls Class.
-     * @return {@code true} if predefined, {@code false} otherwise.
-     */
-    public boolean isPredefinedClass(Class<?> cls) {
-        return predefinedClasses.contains(cls);
-    }
-
-    /**
      * Returns instance of {@link OptimizedMarshaller}.
      *
      * @return Optimized marshaller.
@@ -977,6 +969,7 @@ public class PortableContext implements Externalizable {
             return lowerCaseHashCode(fieldName);
         }
     }
+
     /**
      * Type descriptors.
      */
@@ -1113,21 +1106,27 @@ public class PortableContext implements Externalizable {
      * Type id wrapper.
      */
     static class Type {
-        /** Type id*/
-        private int id;
+        /** Type id */
+        private final int id;
 
         /** Whether the following type is registered in a cache or not */
-        private boolean registered;
+        private final boolean registered;
 
         public Type(int id, boolean registered) {
             this.id = id;
             this.registered = registered;
         }
 
+        /**
+         * @return Type ID.
+         */
         public int id() {
             return id;
         }
 
+        /**
+         * @return Registered flag value.
+         */
         public boolean registered() {
             return registered;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d3c1f07/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java
index 364d5f8..3152c4b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java
@@ -93,9 +93,6 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
     private static final int INIT_CAP = 1024;
 
     /** */
-    private static final ConcurrentHashMap<Class<?>, Boolean> useOptMarshCache = new ConcurrentHashMap<>();
-
-    /** */
     private final PortableContext ctx;
 
     /** */
@@ -197,7 +194,19 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
     void marshal(Object obj, boolean detached) throws PortableException {
         assert obj != null;
 
-        if (useOptimizedMarshaller(obj)) {
+        cls = obj.getClass();
+
+        PortableClassDescriptor desc = ctx.descriptorForClass(cls);
+
+        if (desc == null)
+            throw new PortableException("Object is not portable: [class=" + cls + ']');
+
+        if (desc.excluded()) {
+            doWriteByte(NULL);
+            return;
+        }
+
+        if (desc.useOptimizedMarshaller()) {
             writeByte(OPTM_MARSH);
 
             try {
@@ -214,18 +223,6 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
             return;
         }
 
-        cls = obj.getClass();
-
-        PortableClassDescriptor desc = ctx.descriptorForClass(cls);
-
-        if (desc == null)
-            throw new PortableException("Object is not portable: [class=" + cls + ']');
-
-        if (desc.excluded()) {
-            doWriteByte(NULL);
-            return;
-        }
-
         if (desc.getWriteReplaceMethod() != null) {
             Object replace;
 
@@ -269,44 +266,6 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         desc.write(obj, this);
     }
 
-     /**
-      * Determines whether to use {@link org.apache.ignite.marshaller.optimized.OptimizedMarshaller} for serialization
-      * or not.
-      *
-      * @param obj Object to serialize.
-      * @return {@code true} if to use, {@code false} otherwise.
-      */
-     private boolean useOptimizedMarshaller(Object obj) {
-         Class<?> cls = obj.getClass();
-
-         Boolean use = useOptMarshCache.get(cls);
-
-         if (use != null)
-             return use;
-
-         if (ctx.isPredefinedClass(cls))
-             use = false;
-         else {
-             try {
-                 Method writeObj = cls.getDeclaredMethod("writeObject", ObjectOutputStream.class);
-                 Method readObj = cls.getDeclaredMethod("readObject", ObjectInputStream.class);
-
-                 if (!Modifier.isStatic(writeObj.getModifiers()) && !Modifier.isStatic(readObj.getModifiers()) &&
-                     writeObj.getReturnType() == void.class && readObj.getReturnType() == void.class)
-                     use = true;
-                 else
-                     use = false;
-
-             } catch (NoSuchMethodException e) {
-                 use = false;
-             }
-         }
-
-         useOptMarshCache.putIfAbsent(cls, use);
-
-         return use;
-     }
-
     /**
      * @param obj Object.
      * @return Handle.
@@ -803,12 +762,12 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
             if (tryWriteAsHandle(val))
                 return;
 
-            PortableContext.Type type = ctx.typeId(val.getClass().getComponentType());
+            PortableClassDescriptor desc = ctx.descriptorForClass(val.getClass().getComponentType());
 
             doWriteByte(OBJ_ARR);
 
-            if (type.registered())
-                doWriteInt(type.id());
+            if (desc.registered())
+                doWriteInt(desc.typeId());
             else {
                 doWriteInt(UNREGISTERED_TYPE_ID);
                 doWriteString(val.getClass().getComponentType().getName());
@@ -887,12 +846,12 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
-            PortableContext.Type type = ctx.typeId(val.getClass());
+            PortableClassDescriptor desc = ctx.descriptorForClass(val.getClass());
 
             doWriteByte(ENUM);
 
-            if (type.registered())
-                doWriteInt(type.id());
+            if (desc.registered())
+                doWriteInt(desc.typeId());
             else {
                 doWriteInt(UNREGISTERED_TYPE_ID);
                 doWriteString(val.getClass().getName());
@@ -911,12 +870,11 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
-            PortableContext.Type type = ctx.typeId(val.getClass().getComponentType());
-
+            PortableClassDescriptor desc = ctx.descriptorForClass(val.getClass().getComponentType());
             doWriteByte(ENUM_ARR);
 
-            if (type.registered())
-                doWriteInt(type.id());
+            if (desc.registered())
+                doWriteInt(desc.typeId());
             else {
                 doWriteInt(UNREGISTERED_TYPE_ID);
                 doWriteString(val.getClass().getComponentType().getName());
@@ -937,12 +895,12 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
         if (val == null)
             doWriteByte(NULL);
         else {
-            PortableContext.Type type = ctx.typeId(val);
+            PortableClassDescriptor desc = ctx.descriptorForClass(val);
 
             doWriteByte(CLASS);
 
-            if (type.registered())
-                doWriteInt(type.id());
+            if (desc.registered())
+                doWriteInt(desc.typeId());
             else {
                 doWriteInt(UNREGISTERED_TYPE_ID);
                 doWriteString(val.getClass().getName());

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d3c1f07/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
index c8287a0..4545a58 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.portable;
 
+import java.lang.reflect.Field;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.math.BigDecimal;
@@ -33,6 +34,7 @@ import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.Map;
+import java.util.Set;
 import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.UUID;
@@ -69,7 +71,6 @@ import sun.misc.Unsafe;
 
 import static org.apache.ignite.internal.portable.PortableThreadLocalMemoryAllocator.THREAD_LOCAL_ALLOC;
 import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertTrue;
 
 /**
  * Portable marshaller tests.
@@ -1185,7 +1186,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         customType1.setIdMapper(new PortableIdMapper() {
             @Override public int typeId(String clsName) {
-                return 100;
+                return 300;
             }
 
             @Override public int fieldId(int typeId, String fieldName) {
@@ -1197,7 +1198,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         customType2.setIdMapper(new PortableIdMapper() {
             @Override public int typeId(String clsName) {
-                return 200;
+                return 400;
             }
 
             @Override public int fieldId(int typeId, String fieldName) {
@@ -1209,7 +1210,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         customType3.setIdMapper(new PortableIdMapper() {
             @Override public int typeId(String clsName) {
-                return 300;
+                return 500;
             }
 
             @Override public int fieldId(int typeId, String fieldName) {
@@ -1245,9 +1246,9 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         assertEquals("key".hashCode(), ctx.typeId("Key"));
         assertEquals("nonexistentclass3".hashCode(), ctx.typeId("NonExistentClass3"));
         assertEquals("nonexistentclass4".hashCode(), ctx.typeId("NonExistentClass4"));
-        assertEquals(100, ctx.typeId(getClass().getSimpleName() + "$Value"));
-        assertEquals(200, ctx.typeId("NonExistentClass1"));
-        assertEquals(300, ctx.typeId("NonExistentClass2"));
+        assertEquals(300, ctx.typeId(getClass().getSimpleName() + "$Value"));
+        assertEquals(400, ctx.typeId("NonExistentClass1"));
+        assertEquals(500, ctx.typeId("NonExistentClass2"));
         assertEquals("nonexistentclass5".hashCode(), ctx.typeId("NonExistentClass5"));
     }
 
@@ -1261,16 +1262,16 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         customType1.setIdMapper(new PortableIdMapper() {
             @Override public int typeId(String clsName) {
-                return 100;
+                return 300;
             }
 
             @Override public int fieldId(int typeId, String fieldName) {
                 switch (fieldName) {
                     case "val1":
-                        return 101;
+                        return 301;
 
                     case "val2":
-                        return 102;
+                        return 302;
 
                     default:
                         return 0;
@@ -1282,16 +1283,16 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         customType2.setIdMapper(new PortableIdMapper() {
             @Override public int typeId(String clsName) {
-                return 200;
+                return 400;
             }
 
             @Override public int fieldId(int typeId, String fieldName) {
                 switch (fieldName) {
                     case "val1":
-                        return 201;
+                        return 401;
 
                     case "val2":
-                        return 202;
+                        return 402;
 
                     default:
                         return 0;
@@ -1309,12 +1310,12 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         assertEquals("val".hashCode(), ctx.fieldId("key".hashCode(), "val"));
         assertEquals("val".hashCode(), ctx.fieldId("nonexistentclass2".hashCode(), "val"));
         assertEquals("val".hashCode(), ctx.fieldId("notconfiguredclass".hashCode(), "val"));
-        assertEquals(101, ctx.fieldId(100, "val1"));
-        assertEquals(102, ctx.fieldId(100, "val2"));
-        assertEquals("val3".hashCode(), ctx.fieldId(100, "val3"));
-        assertEquals(201, ctx.fieldId(200, "val1"));
-        assertEquals(202, ctx.fieldId(200, "val2"));
-        assertEquals("val3".hashCode(), ctx.fieldId(200, "val3"));
+        assertEquals(301, ctx.fieldId(300, "val1"));
+        assertEquals(302, ctx.fieldId(300, "val2"));
+        assertEquals("val3".hashCode(), ctx.fieldId(300, "val3"));
+        assertEquals(401, ctx.fieldId(400, "val1"));
+        assertEquals(402, ctx.fieldId(400, "val2"));
+        assertEquals("val3".hashCode(), ctx.fieldId(400, "val3"));
     }
 
     /**
@@ -2243,6 +2244,35 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
+    public void testPredefinedTypeIds() throws Exception {
+        PortableMarshaller marsh = new PortableMarshaller();
+
+        PortableContext pCtx = initPortableContext(marsh);
+
+        Field field = pCtx.getClass().getDeclaredField("predefinedTypeNames");
+
+        field.setAccessible(true);
+
+        Map<String, Integer> map = (Map<String, Integer>)field.get(pCtx);
+
+        assertTrue(map.size() > 0);
+
+        for (Map.Entry<String, Integer> entry : map.entrySet()) {
+            int id = entry.getValue();
+
+            if (id == GridPortableMarshaller.UNREGISTERED_TYPE_ID)
+                continue;
+
+            PortableClassDescriptor desc = pCtx.descriptorForTypeId(false, entry.getValue(), null);
+
+            assertEquals(desc.typeId(), pCtx.typeId(desc.describedClass().getName()));
+            assertEquals(desc.typeId(), pCtx.typeId(pCtx.typeName(desc.describedClass().getName())));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testCyclicReferencesMarshalling() throws Exception {
         PortableMarshaller marsh = new PortableMarshaller();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4d3c1f07/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
index 95dfbe8..349f152 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
@@ -95,11 +95,11 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
             @SuppressWarnings("IfMayBeConditional")
             @Override public int typeId(String clsName) {
                 if (clsName.endsWith("1"))
-                    return 100;
+                    return 300;
                 else if (clsName.endsWith("2"))
-                    return 200;
+                    return 400;
                 else if (clsName.endsWith("InnerClass"))
-                    return 300;
+                    return 500;
                 else
                     return -500;
             }
@@ -120,9 +120,9 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
 
         assertEquals(3, typeMappers.size());
 
-        assertEquals(100, typeMappers.get("GridPortableTestClass1").typeId("GridPortableTestClass1"));
-        assertEquals(200, typeMappers.get("GridPortableTestClass2").typeId("GridPortableTestClass2"));
-        assertEquals(300, typeMappers.get("InnerClass").typeId("InnerClass"));
+        assertEquals(300, typeMappers.get("GridPortableTestClass1").typeId("GridPortableTestClass1"));
+        assertEquals(400, typeMappers.get("GridPortableTestClass2").typeId("GridPortableTestClass2"));
+        assertEquals(500, typeMappers.get("InnerClass").typeId("InnerClass"));
     }
 
     /**
@@ -161,11 +161,11 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
             @SuppressWarnings("IfMayBeConditional")
             @Override public int typeId(String clsName) {
                 if (clsName.endsWith("1"))
-                    return 100;
+                    return 300;
                 else if (clsName.endsWith("2"))
-                    return 200;
+                    return 400;
                 else if (clsName.endsWith("InnerClass"))
-                    return 300;
+                    return 500;
                 else
                     return -500;
             }
@@ -186,9 +186,9 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
 
         assertEquals(3, typeMappers.size());
 
-        assertEquals(100, typeMappers.get("GridPortableTestClass1").typeId("GridPortableTestClass1"));
-        assertEquals(200, typeMappers.get("GridPortableTestClass2").typeId("GridPortableTestClass2"));
-        assertEquals(300, typeMappers.get("InnerClass").typeId("InnerClass"));
+        assertEquals(300, typeMappers.get("GridPortableTestClass1").typeId("GridPortableTestClass1"));
+        assertEquals(400, typeMappers.get("GridPortableTestClass2").typeId("GridPortableTestClass2"));
+        assertEquals(500, typeMappers.get("InnerClass").typeId("InnerClass"));
     }
 
     /**
@@ -203,11 +203,11 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
             @SuppressWarnings("IfMayBeConditional")
             @Override public int typeId(String clsName) {
                 if (clsName.endsWith("1"))
-                    return 100;
+                    return 300;
                 else if (clsName.endsWith("2"))
-                    return 200;
+                    return 400;
                 else if (clsName.endsWith("InnerClass"))
-                    return 300;
+                    return 500;
                 else
                     return -500;
             }
@@ -228,9 +228,9 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
 
         assertEquals(3, typeMappers.size());
 
-        assertEquals(100, typeMappers.get("GridPortableTestClass1").typeId("GridPortableTestClass1"));
-        assertEquals(200, typeMappers.get("GridPortableTestClass2").typeId("GridPortableTestClass2"));
-        assertEquals(300, typeMappers.get("InnerClass").typeId("InnerClass"));
+        assertEquals(300, typeMappers.get("GridPortableTestClass1").typeId("GridPortableTestClass1"));
+        assertEquals(400, typeMappers.get("GridPortableTestClass2").typeId("GridPortableTestClass2"));
+        assertEquals(500, typeMappers.get("InnerClass").typeId("InnerClass"));
     }
 
     /**
@@ -310,9 +310,9 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
             @SuppressWarnings("IfMayBeConditional")
             @Override public int typeId(String clsName) {
                 if (clsName.endsWith("1"))
-                    return 100;
+                    return 300;
                 else if (clsName.endsWith("2"))
-                    return 200;
+                    return 400;
                 else
                     return -500;
             }
@@ -333,8 +333,8 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
 
         assertEquals(3, typeMappers.size());
 
-        assertEquals(100, typeMappers.get("GridPortableTestClass1").typeId("GridPortableTestClass1"));
-        assertEquals(200, typeMappers.get("GridPortableTestClass2").typeId("GridPortableTestClass2"));
+        assertEquals(300, typeMappers.get("GridPortableTestClass1").typeId("GridPortableTestClass1"));
+        assertEquals(400, typeMappers.get("GridPortableTestClass2").typeId("GridPortableTestClass2"));
     }
 
     /**
@@ -372,9 +372,9 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
             @SuppressWarnings("IfMayBeConditional")
             @Override public int typeId(String clsName) {
                 if (clsName.endsWith("1"))
-                    return 100;
+                    return 300;
                 else if (clsName.endsWith("2"))
-                    return 200;
+                    return 400;
                 else
                     return -500;
             }
@@ -395,8 +395,8 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
 
         assertEquals(3, typeMappers.size());
 
-        assertEquals(100, typeMappers.get("GridPortableTestClass1").typeId("GridPortableTestClass1"));
-        assertEquals(200, typeMappers.get("GridPortableTestClass2").typeId("GridPortableTestClass2"));
+        assertEquals(300, typeMappers.get("GridPortableTestClass1").typeId("GridPortableTestClass1"));
+        assertEquals(400, typeMappers.get("GridPortableTestClass2").typeId("GridPortableTestClass2"));
     }
 
     /**
@@ -411,9 +411,9 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
             @SuppressWarnings("IfMayBeConditional")
             @Override public int typeId(String clsName) {
                 if (clsName.endsWith("1"))
-                    return 100;
+                    return 300;
                 else if (clsName.endsWith("2"))
-                    return 200;
+                    return 400;
                 else
                     return -500;
             }
@@ -434,8 +434,8 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
 
         assertEquals(3, typeMappers.size());
 
-        assertEquals(100, typeMappers.get("GridPortableTestClass1").typeId("GridPortableTestClass1"));
-        assertEquals(200, typeMappers.get("GridPortableTestClass2").typeId("GridPortableTestClass2"));
+        assertEquals(300, typeMappers.get("GridPortableTestClass1").typeId("GridPortableTestClass1"));
+        assertEquals(400, typeMappers.get("GridPortableTestClass2").typeId("GridPortableTestClass2"));
     }
 
     /**


[02/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/TestRunner.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/TestRunner.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/TestRunner.cs
index 73c9bcb..2b0ab8e 100644
--- a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/TestRunner.cs
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/TestRunner.cs
@@ -20,6 +20,8 @@ namespace Apache.Ignite.Core.Tests
     using System;
     using System.Diagnostics;
     using System.Reflection;
+    using Apache.Ignite.Core.Tests.Memory;
+    using NUnit.ConsoleRunner;
 
     public static class TestRunner
     {
@@ -31,16 +33,15 @@ namespace Apache.Ignite.Core.Tests
 
             //TestOne(typeof(ContinuousQueryAtomiclBackupTest), "TestInitialQuery");
 
-            //TestAll(typeof(IgnitionTest));
-
-            TestAllInAssembly();
+            TestAll(typeof (ExecutableTest));
+            //TestAllInAssembly();
         }
 
         private static void TestOne(Type testClass, string method)
         {
             string[] args = { "/run:" + testClass.FullName + "." + method, Assembly.GetAssembly(testClass).Location };
 
-            int returnCode = NUnit.ConsoleRunner.Runner.Main(args);
+            int returnCode = Runner.Main(args);
 
             if (returnCode != 0)
                 Console.Beep();
@@ -50,7 +51,7 @@ namespace Apache.Ignite.Core.Tests
         {
             string[] args = { "/run:" + testClass.FullName, Assembly.GetAssembly(testClass).Location };
 
-            int returnCode = NUnit.ConsoleRunner.Runner.Main(args);
+            int returnCode = Runner.Main(args);
 
             if (returnCode != 0)
                 Console.Beep();
@@ -58,9 +59,9 @@ namespace Apache.Ignite.Core.Tests
 
         private static void TestAllInAssembly()
         {
-            string[] args = { Assembly.GetAssembly(typeof(IgnitionTest)).Location };
+            string[] args = { Assembly.GetAssembly(typeof(InteropMemoryTest)).Location };
 
-            int returnCode = NUnit.ConsoleRunner.Runner.Main(args);
+            int returnCode = Runner.Main(args);
 
             if (returnCode != 0)
                 Console.Beep();

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
new file mode 100644
index 0000000..ca45d68
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
@@ -0,0 +1,292 @@
+/*
+ * 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.Collections.Concurrent;
+    using System.Collections.Generic;
+    using System.Linq;
+    using System.Threading;
+    using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.Tests.Process;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Test utility methods.
+    /// </summary>
+    public static class TestUtils
+    {
+        /** Indicates long running and/or memory/cpu intensive test. */
+        public const string CategoryIntensive = "LONG_TEST";
+
+        /** */
+        public const int DfltBusywaitSleepInterval = 200;
+
+        /** */
+
+        private static readonly IList<string> TestJvmOpts = Environment.Is64BitProcess
+            ? new List<string>
+            {
+                "-XX:+HeapDumpOnOutOfMemoryError",
+                "-Xms1g",
+                "-Xmx4g",
+                "-ea"
+            }
+            : new List<string>
+            {
+                "-XX:+HeapDumpOnOutOfMemoryError",
+                "-Xms512m",
+                "-Xmx512m",
+                "-ea",
+                "-DIGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE=1000"
+            };
+
+        /** */
+        private static readonly IList<string> JvmDebugOpts =
+            new List<string> { "-Xdebug", "-Xnoagent", "-Djava.compiler=NONE", "-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005" };
+
+        /** */
+        public static bool JvmDebug = true;
+
+        /** */
+        [ThreadStatic]
+        private static Random _random;
+
+        /** */
+        private static int _seed = Environment.TickCount;
+
+        /// <summary>
+        /// Kill GridGain processes.
+        /// </summary>
+        public static void KillProcesses()
+        {
+            IgniteProcess.KillAll();
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        public static Random Random
+        {
+            get { return _random ?? (_random = new Random(Interlocked.Increment(ref _seed))); }
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        /// <returns></returns>
+        public static IList<string> TestJavaOptions()
+        {
+            IList<string> ops = new List<string>(TestJvmOpts);
+
+            if (JvmDebug)
+            {
+                foreach (string opt in JvmDebugOpts)
+                    ops.Add(opt);
+            }
+
+            return ops;
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        /// <returns></returns>
+        public static string CreateTestClasspath()
+        {
+            return IgniteManager.CreateClasspath(forceTestClasspath: true);
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        /// <param name="action"></param>
+        /// <param name="threadNum"></param>
+        public static void RunMultiThreaded(Action action, int threadNum)
+        {
+            List<Thread> threads = new List<Thread>(threadNum);
+
+            var errors = new ConcurrentBag<Exception>();
+
+            for (int i = 0; i < threadNum; i++)
+            {
+                threads.Add(new Thread(() =>
+                {
+                    try
+                    {
+                        action();
+                    }
+                    catch (Exception e)
+                    {
+                        errors.Add(e);
+                    }
+                }));
+            }
+
+            foreach (Thread thread in threads)
+                thread.Start();
+
+            foreach (Thread thread in threads)
+                thread.Join();
+            
+            foreach (var ex in errors)
+                Assert.Fail("Unexpected exception: " + ex);
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        /// <param name="action"></param>
+        /// <param name="threadNum"></param>
+        /// <param name="duration">Duration of test execution in seconds</param>
+        public static void RunMultiThreaded(Action action, int threadNum, int duration)
+        {
+            List<Thread> threads = new List<Thread>(threadNum);
+
+            var errors = new ConcurrentBag<Exception>();
+
+            bool stop = false;
+
+            for (int i = 0; i < threadNum; i++)
+            {
+                threads.Add(new Thread(() =>
+                {
+                    try
+                    {
+                        while (true)
+                        {
+                            Thread.MemoryBarrier();
+
+                            // ReSharper disable once AccessToModifiedClosure
+                            if (stop)
+                                break;
+
+                            action();
+                        }
+                    }
+                    catch (Exception e)
+                    {
+                        errors.Add(e);
+                    }
+                }));
+            }
+
+            foreach (Thread thread in threads)
+                thread.Start();
+
+            Thread.Sleep(duration * 1000);
+
+            stop = true;
+
+            Thread.MemoryBarrier();
+
+            foreach (Thread thread in threads)
+                thread.Join();
+
+            foreach (var ex in errors)
+                Assert.Fail("Unexpected exception: " + ex);
+        }
+
+        /// <summary>
+        /// Wait for particular topology size.
+        /// </summary>
+        /// <param name="grid">Grid.</param>
+        /// <param name="size">Size.</param>
+        /// <param name="timeout">Timeout.</param>
+        /// <returns>
+        ///   <c>True</c> if topology took required size.
+        /// </returns>
+        public static bool WaitTopology(this IIgnite grid, int size, int timeout)
+        {
+            int left = timeout;
+
+            while (true)
+            {
+                if (grid.Cluster.Nodes().Count != size)
+                {
+                    if (left > 0)
+                    {
+                        Thread.Sleep(100);
+
+                        left -= 100;
+                    }
+                    else
+                        break;
+                }
+                else
+                    return true;
+            }
+
+            return false;
+        }
+
+        /// <summary>
+        /// Asserts that the handle registry is empty.
+        /// </summary>
+        /// <param name="timeout">Timeout, in milliseconds.</param>
+        /// <param name="grids">Grids to check.</param>
+        public static void AssertHandleRegistryIsEmpty(int timeout, params IIgnite[] grids)
+        {
+            foreach (var g in grids)
+                AssertHandleRegistryIsEmpty(g, timeout);
+        }
+
+        /// <summary>
+        /// Asserts that the handle registry is empty.
+        /// </summary>
+        /// <param name="grid">The grid to check.</param>
+        /// <param name="timeout">Timeout, in milliseconds.</param>
+        public static void AssertHandleRegistryIsEmpty(IIgnite grid, int timeout)
+        {
+            var handleRegistry = ((Ignite)grid).HandleRegistry;
+
+            if (WaitForCondition(() => handleRegistry.Count == 0, timeout))
+                return;
+
+            var items = handleRegistry.GetItems();
+
+            if (items.Any())
+                Assert.Fail("HandleRegistry is not empty in grid '{0}':\n '{1}'", grid.Name,
+                    items.Select(x => x.ToString()).Aggregate((x, y) => x + "\n" + y));
+        }
+
+        /// <summary>
+        /// Waits for condition, polling in busy wait loop.
+        /// </summary>
+        /// <param name="cond">Condition.</param>
+        /// <param name="timeout">Timeout, in milliseconds.</param>
+        /// <returns>True if condition predicate returned true within interval; false otherwise.</returns>
+        public static bool WaitForCondition(Func<bool> cond, int timeout)
+        {
+            if (timeout <= 0)
+                return cond();
+
+            var maxTime = DateTime.Now.AddMilliseconds(timeout + DfltBusywaitSleepInterval);
+
+            while (DateTime.Now < maxTime)
+            {
+                if (cond())
+                    return true;
+
+                Thread.Sleep(DfltBusywaitSleepInterval);
+            }
+
+            return false;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/TypeResolverTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/TypeResolverTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/TypeResolverTest.cs
new file mode 100644
index 0000000..a49ee1b
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/TypeResolverTest.cs
@@ -0,0 +1,107 @@
+/*
+ * 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.Collections.Generic;
+    using System.Linq;
+    using System.Reflection;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Tests.TestDll;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// <see cref="TypeResolver"/> tests.
+    /// </summary>
+    public class TypeResolverTest
+    {
+        /// <summary>
+        /// Tests generic type resolve.
+        /// </summary>
+        [Test]
+        public void TestGenerics()
+        {
+            var testTypes = new[]
+            {
+                typeof (TestGenericPortable<int>),
+                typeof (TestGenericPortable<string>),
+                typeof (TestGenericPortable<TestGenericPortable<int>>),
+                typeof (TestGenericPortable<List<Tuple<int, string>>>),
+                typeof (TestGenericPortable<List<TestGenericPortable<List<Tuple<int, string>>>>>),
+                typeof (List<TestGenericPortable<List<TestGenericPortable<List<Tuple<int, string>>>>>>),
+                typeof (TestGenericPortable<int, string>),
+                typeof (TestGenericPortable<int, TestGenericPortable<string>>),
+                typeof (TestGenericPortable<int, string, Type>),
+                typeof (TestGenericPortable<int, string, TestGenericPortable<int, string, Type>>)
+            };
+
+            foreach (var type in testTypes)
+            {
+                // Without assembly
+                var resolvedType = new TypeResolver().ResolveType(type.FullName);
+                Assert.AreEqual(type.FullName, resolvedType.FullName);
+                
+                // With assembly
+                resolvedType = new TypeResolver().ResolveType(type.FullName, type.Assembly.FullName);
+                Assert.AreEqual(type.FullName, resolvedType.FullName);
+
+                // Assembly-qualified
+                resolvedType = new TypeResolver().ResolveType(type.AssemblyQualifiedName);
+                Assert.AreEqual(type.FullName, resolvedType.FullName);
+            }
+        }
+
+        /// <summary>
+        /// Tests loading a type from referenced assembly that is not yet loaded.
+        /// </summary>
+        [Test]
+        public void TestReferencedAssemblyLoading()
+        {
+            const string dllName = "Apache.Ignite.Core.Tests.TestDll";
+
+            const string typeName = "Apache.Ignite.Core.Tests.TestDll.TestClass";
+
+            // Check that the dll is not yet loaded
+            Assert.IsFalse(AppDomain.CurrentDomain.GetAssemblies().Any(x => x.FullName.StartsWith(dllName)));
+
+            // Check that the dll is referenced by current assembly
+            Assert.IsTrue(Assembly.GetExecutingAssembly().GetReferencedAssemblies()
+                .Any(x => x.FullName.StartsWith(dllName)));
+
+            // Check resolver
+            var type = new TypeResolver().ResolveType(typeName);
+            
+            Assert.IsNotNull(type);
+            Assert.AreEqual(typeName, type.FullName);
+            Assert.IsNotNull(Activator.CreateInstance(type));
+
+            // At this moment the dll should be loaded
+            Assert.IsTrue(AppDomain.CurrentDomain.GetAssemblies().Any(x => x.FullName.StartsWith(dllName)));
+        }
+
+        /// <summary>
+        /// Unused method that forces C# compiler to include TestDll assembly reference.
+        /// Without this, compiler will remove the reference as unused.
+        /// However, since it is never called, TestDll does not get loaded.
+        /// </summary>
+        public void UnusedMethod()
+        {
+            Assert.IsNotNull(typeof(TestClass));
+        }        
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputeBroadcastTask.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputeBroadcastTask.java b/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputeBroadcastTask.java
new file mode 100644
index 0000000..c721e16
--- /dev/null
+++ b/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputeBroadcastTask.java
@@ -0,0 +1,73 @@
+/*
+ * 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.platform;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.compute.ComputeJob;
+import org.apache.ignite.compute.ComputeJobAdapter;
+import org.apache.ignite.compute.ComputeJobResult;
+import org.apache.ignite.compute.ComputeTaskAdapter;
+import org.apache.ignite.resources.IgniteInstanceResource;
+import org.jetbrains.annotations.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+/**
+ * Task collecting IDs of all nodes where it was executed.
+ */
+public class PlatformComputeBroadcastTask extends ComputeTaskAdapter<Object, Collection<UUID>> {
+    /** {@inheritDoc} */
+    @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Object arg) {
+        Map<ComputeJob, ClusterNode> jobs = new HashMap<>();
+
+        for (ClusterNode node : subgrid)
+            jobs.put(new BroadcastJob(), node);
+
+        return jobs;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Collection<UUID> reduce(List<ComputeJobResult> results) {
+        List<UUID> ids = new ArrayList<>();
+
+        for (ComputeJobResult res : results)
+            ids.add((UUID)res.getData());
+
+        return ids;
+    }
+
+    /**
+     * Job.
+     */
+    private static class BroadcastJob extends ComputeJobAdapter {
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /** {@inheritDoc} */
+        @Nullable @Override public Object execute() {
+            return ignite.cluster().localNode().id();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputeDecimalTask.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputeDecimalTask.java b/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputeDecimalTask.java
new file mode 100644
index 0000000..a14e481
--- /dev/null
+++ b/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputeDecimalTask.java
@@ -0,0 +1,106 @@
+/*
+ * 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.platform;
+
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.compute.ComputeJob;
+import org.apache.ignite.compute.ComputeJobAdapter;
+import org.apache.ignite.compute.ComputeJobResult;
+import org.apache.ignite.compute.ComputeTaskAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.jetbrains.annotations.Nullable;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.math.BigDecimal;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+@SuppressWarnings({"ConstantConditions", "UnusedDeclaration"})
+public class PlatformComputeDecimalTask extends ComputeTaskAdapter<Object[], BigDecimal> {
+    /** {@inheritDoc} */
+    @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Object[] arg) {
+        return Collections.singletonMap(new DecimalJob((BigDecimal)arg[0], (String)arg[1]), F.first(subgrid));
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+    @Nullable @Override public BigDecimal reduce(List<ComputeJobResult> results) {
+        ComputeJobResult res = results.get(0);
+
+        if (res.getException() != null)
+            throw res.getException();
+        else
+            return results.get(0).getData();
+    }
+
+    /**
+     * Job.
+     */
+    private static class DecimalJob extends ComputeJobAdapter implements Externalizable {
+        /** Value. */
+        private BigDecimal val;
+
+        /** Value as string. */
+        private String valStr;
+
+        /**
+         * Constructor.
+         */
+        public DecimalJob() {
+            // No-op.
+        }
+
+        /**
+         * Constructor.
+         *
+         * @param val Value.
+         * @param valStr Value as string.
+         */
+        private DecimalJob(BigDecimal val, String valStr) {
+            this.val = val;
+            this.valStr = valStr;
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public Object execute() {
+            BigDecimal exp = new BigDecimal(valStr.replace(',', '.'));
+
+            if (val != null && !exp.equals(val))
+                throw new IgniteException("Actual=" + val);
+
+            return exp;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            out.writeObject(val);
+            out.writeObject(valStr);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            val = (BigDecimal)in.readObject();
+            valStr = (String)in.readObject();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputeEchoTask.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputeEchoTask.java b/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputeEchoTask.java
new file mode 100644
index 0000000..f64ca7d
--- /dev/null
+++ b/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputeEchoTask.java
@@ -0,0 +1,188 @@
+/*
+ * 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.platform;
+
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.compute.ComputeJob;
+import org.apache.ignite.compute.ComputeJobAdapter;
+import org.apache.ignite.compute.ComputeJobResult;
+import org.apache.ignite.compute.ComputeTaskAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.jetbrains.annotations.Nullable;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Test task producing result without any arguments.
+ */
+public class PlatformComputeEchoTask extends ComputeTaskAdapter<Integer, Object> {
+    /** Type: NULL. */
+    private static final int TYPE_NULL = 0;
+
+    /** Type: byte. */
+    private static final int TYPE_BYTE = 1;
+
+    /** Type: bool. */
+    private static final int TYPE_BOOL = 2;
+
+    /** Type: short. */
+    private static final int TYPE_SHORT = 3;
+
+    /** Type: char. */
+    private static final int TYPE_CHAR = 4;
+
+    /** Type: int. */
+    private static final int TYPE_INT = 5;
+
+    /** Type: long. */
+    private static final int TYPE_LONG = 6;
+
+    /** Type: float. */
+    private static final int TYPE_FLOAT = 7;
+
+    /** Type: double. */
+    private static final int TYPE_DOUBLE = 8;
+
+    /** Type: array. */
+    private static final int TYPE_ARRAY = 9;
+
+    /** Type: collection. */
+    private static final int TYPE_COLLECTION = 10;
+
+    /** Type: map. */
+    private static final int TYPE_MAP = 11;
+
+    /** Type: portable object which exists in all platforms. */
+    private static final int TYPE_PORTABLE = 12;
+
+    /** Type: portable object which exists only in Java. */
+    private static final int TYPE_PORTABLE_JAVA = 13;
+
+    /** Type: object array. */
+    private static final int TYPE_OBJ_ARRAY = 14;
+
+    /** Type: portable object array. */
+    private static final int TYPE_PORTABLE_ARRAY = 15;
+
+    /** Type: enum. */
+    private static final int TYPE_ENUM = 16;
+
+    /** Type: enum array. */
+    private static final int TYPE_ENUM_ARRAY = 17;
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid,
+        @Nullable Integer arg) {
+        return Collections.singletonMap(new EchoJob(arg), F.first(subgrid));
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Object reduce(List<ComputeJobResult> results) {
+        return results.get(0).getData();
+    }
+
+    /**
+     * Job.
+     */
+    private static class EchoJob extends ComputeJobAdapter {
+        /** Type. */
+        private Integer type;
+
+        /**
+         * Constructor.
+         *
+         * @param type Result type.
+         */
+        public EchoJob(Integer type) {
+            this.type = type;
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public Object execute() {
+            switch (type) {
+                case TYPE_NULL:
+                    return null;
+
+                case TYPE_BYTE:
+                    return (byte)1;
+
+                case TYPE_BOOL:
+                    return true;
+
+                case TYPE_SHORT:
+                    return (short)1;
+
+                case TYPE_CHAR:
+                    return (char)1;
+
+                case TYPE_INT:
+                    return 1;
+
+                case TYPE_LONG:
+                    return (long)1;
+
+                case TYPE_FLOAT:
+                    return (float)1;
+
+                case TYPE_DOUBLE:
+                    return (double)1;
+
+                case TYPE_ARRAY:
+                    return new int[] { 1 };
+
+                case TYPE_COLLECTION:
+                    return Collections.singletonList(1);
+
+                case TYPE_MAP:
+                    return Collections.singletonMap(1, 1);
+
+                case TYPE_PORTABLE:
+                    return new PlatformComputePortable(1);
+
+                case TYPE_PORTABLE_JAVA:
+                    return new PlatformComputeJavaPortable(1);
+
+                case TYPE_OBJ_ARRAY:
+                    return new String[] { "foo", "bar", "baz" };
+
+                case TYPE_PORTABLE_ARRAY:
+                    return new PlatformComputePortable[] {
+                        new PlatformComputePortable(1),
+                        new PlatformComputePortable(2),
+                        new PlatformComputePortable(3)
+                    };
+
+                case TYPE_ENUM:
+                    return PlatformComputeEnum.BAR;
+
+                case TYPE_ENUM_ARRAY:
+                    return new PlatformComputeEnum[] {
+                        PlatformComputeEnum.BAR,
+                        PlatformComputeEnum.BAZ,
+                        PlatformComputeEnum.FOO
+                    };
+
+                default:
+                    throw new IgniteException("Unknown type: " + type);
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputeEnum.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputeEnum.java b/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputeEnum.java
new file mode 100644
index 0000000..7fc0623
--- /dev/null
+++ b/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputeEnum.java
@@ -0,0 +1,28 @@
+/*
+ * 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.platform;
+
+/**
+ * Test enum.
+ */
+public enum PlatformComputeEnum
+{
+    FOO,
+    BAR,
+    BAZ
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputeJavaPortable.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputeJavaPortable.java b/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputeJavaPortable.java
new file mode 100644
index 0000000..7a940c4
--- /dev/null
+++ b/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputeJavaPortable.java
@@ -0,0 +1,39 @@
+/*
+ * 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.platform;
+
+/**
+ * Portable object defined only in Java.
+ */
+public class PlatformComputeJavaPortable extends PlatformComputePortable {
+    /**
+     * Constructor.
+     */
+    public PlatformComputeJavaPortable() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param field Field.
+     */
+    public PlatformComputeJavaPortable(int field) {
+        super(field);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputePortable.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputePortable.java b/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputePortable.java
new file mode 100644
index 0000000..f31f093
--- /dev/null
+++ b/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputePortable.java
@@ -0,0 +1,42 @@
+/*
+ * 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.platform;
+
+/**
+ * Portable object for task tests.
+ */
+public class PlatformComputePortable {
+    /** Field. */
+    public int field;
+
+    /**
+     * Constructor.
+     */
+    public PlatformComputePortable() {
+        // No-op.
+    }
+
+    /**
+     * Constructor,
+     *
+     * @param field Field.
+     */
+    public PlatformComputePortable(int field) {
+        this.field = field;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputePortableArgTask.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputePortableArgTask.java b/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputePortableArgTask.java
new file mode 100644
index 0000000..0e8b825
--- /dev/null
+++ b/modules/platform/src/test/java/org/apache/ignite/platform/PlatformComputePortableArgTask.java
@@ -0,0 +1,119 @@
+/*
+ * 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.platform;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.compute.ComputeJob;
+import org.apache.ignite.compute.ComputeJobAdapter;
+import org.apache.ignite.compute.ComputeJobResult;
+import org.apache.ignite.compute.ComputeTaskAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.portable.PortableMetadata;
+import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.resources.IgniteInstanceResource;
+import org.jetbrains.annotations.Nullable;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Task working with portable argument.
+ */
+public class PlatformComputePortableArgTask extends ComputeTaskAdapter<Object, Integer> {
+    /** {@inheritDoc} */
+    @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Object arg) {
+        return Collections.singletonMap(new PortableArgJob(arg), F.first(subgrid));
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+    @Nullable @Override public Integer reduce(List<ComputeJobResult> results) {
+        ComputeJobResult res = results.get(0);
+
+        if (res.getException() != null)
+            throw res.getException();
+        else
+            return results.get(0).getData();
+    }
+
+    /**
+     * Job.
+     */
+    private static class PortableArgJob extends ComputeJobAdapter implements Externalizable {
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /** Argument. */
+        private Object arg;
+
+        /**
+         * Constructor.
+         */
+        public PortableArgJob() {
+            // No-op.
+        }
+
+        /**
+         * Constructor.
+         *
+         * @param arg Argument.
+         */
+        private PortableArgJob(Object arg) {
+            this.arg = arg;
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public Object execute() {
+            PortableObject arg0 = ((PortableObject)arg);
+
+            PortableMetadata meta = ignite.portables().metadata(arg0.typeId());
+
+            if (meta == null)
+                throw new IgniteException("Metadata doesn't exist.");
+
+            if (meta.fields() == null || !meta.fields().contains("Field"))
+                throw new IgniteException("Field metadata doesn't exist.");
+
+            if (!F.eq("int", meta.fieldTypeName("Field")))
+                throw new IgniteException("Invalid field type: " + meta.fieldTypeName("Field"));
+
+            if (meta.affinityKeyFieldName() != null)
+                throw new IgniteException("Unexpected affinity key: " + meta.affinityKeyFieldName());
+
+            return arg0.field("field");
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            out.writeObject(arg);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            arg = in.readObject();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/java/org/apache/ignite/platform/PlatformEventsWriteEventTask.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/java/org/apache/ignite/platform/PlatformEventsWriteEventTask.java b/modules/platform/src/test/java/org/apache/ignite/platform/PlatformEventsWriteEventTask.java
new file mode 100644
index 0000000..d9dee9d
--- /dev/null
+++ b/modules/platform/src/test/java/org/apache/ignite/platform/PlatformEventsWriteEventTask.java
@@ -0,0 +1,146 @@
+/*
+ * 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.platform;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.compute.ComputeJob;
+import org.apache.ignite.compute.ComputeJobAdapter;
+import org.apache.ignite.compute.ComputeJobResult;
+import org.apache.ignite.compute.ComputeTaskAdapter;
+import org.apache.ignite.events.CacheEvent;
+import org.apache.ignite.events.CacheQueryExecutedEvent;
+import org.apache.ignite.events.CacheQueryReadEvent;
+import org.apache.ignite.events.CacheRebalancingEvent;
+import org.apache.ignite.events.CheckpointEvent;
+import org.apache.ignite.events.DiscoveryEvent;
+import org.apache.ignite.events.JobEvent;
+import org.apache.ignite.events.SwapSpaceEvent;
+import org.apache.ignite.events.TaskEvent;
+import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
+import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
+import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.events.*;
+
+import org.apache.ignite.plugin.security.SecurityPermission;
+import org.apache.ignite.plugin.security.SecuritySubjectType;
+import org.apache.ignite.resources.IgniteInstanceResource;
+import org.jetbrains.annotations.Nullable;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+/**
+ * Test task writing all events to a stream.
+ */
+@SuppressWarnings("UnusedDeclaration")
+public class PlatformEventsWriteEventTask extends ComputeTaskAdapter<Long, Object> {
+    /** {@inheritDoc} */
+    @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid,
+        Long ptr) {
+        return Collections.singletonMap(new Job(ptr, F.first(subgrid)), F.first(subgrid));
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Object reduce(List<ComputeJobResult> results) {
+        return results.get(0).getData();
+    }
+
+    /**
+     * Job.
+     */
+    @SuppressWarnings("deprecation")
+    private static class Job extends ComputeJobAdapter {
+        /** Grid. */
+        @IgniteInstanceResource
+        protected transient Ignite ignite;
+
+        /** Stream ptr. */
+        private final long ptr;
+
+        private final ClusterNode node;
+
+        /**
+         * Constructor.
+         *
+         * @param ptr Stream ptr.
+         */
+        public Job(long ptr, ClusterNode node) {
+            this.ptr = ptr;
+            this.node = node;
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public Object execute() {
+            PlatformContext ctx = PlatformUtils.platformContext(ignite);
+
+            try (PlatformMemory mem = ctx.memory().get(ptr)) {
+                PlatformOutputStream out = mem.output();
+                PortableRawWriterEx writer = ctx.writer(out);
+
+                int evtType = EventType.EVT_SWAP_SPACE_CLEARED;
+                String msg = "msg";
+                UUID uuid = new UUID(1, 2);
+                IgniteUuid igniteUuid = new IgniteUuid(uuid, 3);
+
+                ctx.writeEvent(writer, new CacheEvent("cacheName", node, node, "msg", evtType, 1, true, 2,
+                    igniteUuid, 3, 4, true, 5, true, uuid, "cloClsName", "taskName"));
+
+                //noinspection unchecked
+                ctx.writeEvent(writer, new CacheQueryExecutedEvent(node, msg, evtType, "qryType", "cacheName",
+                    "clsName", "clause", null, null, null, uuid, "taskName"));
+
+                //noinspection unchecked
+                ctx.writeEvent(writer, new CacheQueryReadEvent(node, msg, evtType, "qryType", "cacheName",
+                    "clsName", "clause", null, null, null, uuid, "taskName", 1, 2, 3, 4));
+
+                ctx.writeEvent(writer, new CacheRebalancingEvent("cacheName", node, msg, evtType, 1, node, 2, 3));
+
+                ctx.writeEvent(writer, new CheckpointEvent(node, msg, evtType, "cpKey"));
+
+                DiscoveryEvent discoveryEvent = new DiscoveryEvent(node, msg, evtType, node);
+                discoveryEvent.topologySnapshot(ignite.cluster().topologyVersion(), ignite.cluster().nodes());
+                ctx.writeEvent(writer, discoveryEvent);
+
+                JobEvent jobEvent = new JobEvent(node, msg, evtType);
+                jobEvent.jobId(igniteUuid);
+                jobEvent.taskClassName("taskClsName");
+                jobEvent.taskName("taskName");
+                jobEvent.taskNode(node);
+                jobEvent.taskSessionId(igniteUuid);
+                jobEvent.taskSubjectId(uuid);
+                ctx.writeEvent(writer, jobEvent);
+
+                ctx.writeEvent(writer, new SwapSpaceEvent(node, msg, evtType, "space"));
+
+                ctx.writeEvent(writer, new TaskEvent(node, msg, evtType, igniteUuid, "taskName", "taskClsName",
+                    true, uuid));
+
+                out.synchronize();
+            }
+
+            return true;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/java/org/apache/ignite/platform/PlatformMaxMemoryTask.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/java/org/apache/ignite/platform/PlatformMaxMemoryTask.java b/modules/platform/src/test/java/org/apache/ignite/platform/PlatformMaxMemoryTask.java
new file mode 100644
index 0000000..6effb0f
--- /dev/null
+++ b/modules/platform/src/test/java/org/apache/ignite/platform/PlatformMaxMemoryTask.java
@@ -0,0 +1,57 @@
+/*
+ * 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.platform;
+
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.compute.ComputeJob;
+import org.apache.ignite.compute.ComputeJobAdapter;
+import org.apache.ignite.compute.ComputeJobResult;
+import org.apache.ignite.compute.ComputeTaskAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.jetbrains.annotations.Nullable;
+
+import java.lang.management.ManagementFactory;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Task to get max Java heap memory for node.
+ */
+public class PlatformMaxMemoryTask extends ComputeTaskAdapter<Object, Long> {
+    /** {@inheritDoc} */
+    @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid,
+        @Nullable Object arg) {
+        return Collections.singletonMap(new MaxMemoryJob(), F.first(subgrid));
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Long reduce(List<ComputeJobResult> results) {
+        return results.get(0).getData();
+    }
+
+    /**
+     * Job.
+     */
+    private static class MaxMemoryJob extends ComputeJobAdapter {
+        /** {@inheritDoc} */
+        @Nullable @Override public Object execute() {
+            return ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getMax();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/java/org/apache/ignite/platform/PlatformMinMemoryTask.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/java/org/apache/ignite/platform/PlatformMinMemoryTask.java b/modules/platform/src/test/java/org/apache/ignite/platform/PlatformMinMemoryTask.java
new file mode 100644
index 0000000..23292f7
--- /dev/null
+++ b/modules/platform/src/test/java/org/apache/ignite/platform/PlatformMinMemoryTask.java
@@ -0,0 +1,57 @@
+/*
+ * 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.platform;
+
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.compute.ComputeJob;
+import org.apache.ignite.compute.ComputeJobAdapter;
+import org.apache.ignite.compute.ComputeJobResult;
+import org.apache.ignite.compute.ComputeTaskAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.jetbrains.annotations.Nullable;
+
+import java.lang.management.ManagementFactory;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Task to get min Java heap memory for node.
+ */
+public class PlatformMinMemoryTask extends ComputeTaskAdapter<Object, Long> {
+    /** {@inheritDoc} */
+    @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid,
+        @Nullable Object arg) {
+        return Collections.singletonMap(new MinMemoryJob(), F.first(subgrid));
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Long reduce(List<ComputeJobResult> results) {
+        return results.get(0).getData();
+    }
+
+    /**
+     * Job.
+     */
+    private static class MinMemoryJob extends ComputeJobAdapter {
+        /** {@inheritDoc} */
+        @Nullable @Override public Object execute() {
+            return ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getInit();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/java/org/apache/ignite/platform/lifecycle/PlatformJavaLifecycleBean.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/java/org/apache/ignite/platform/lifecycle/PlatformJavaLifecycleBean.java b/modules/platform/src/test/java/org/apache/ignite/platform/lifecycle/PlatformJavaLifecycleBean.java
new file mode 100644
index 0000000..d60912b
--- /dev/null
+++ b/modules/platform/src/test/java/org/apache/ignite/platform/lifecycle/PlatformJavaLifecycleBean.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.platform.lifecycle;
+
+import org.apache.ignite.lifecycle.LifecycleBean;
+import org.apache.ignite.lifecycle.LifecycleEventType;
+
+/**
+ * Java lifecycle bean.
+ */
+public class PlatformJavaLifecycleBean implements LifecycleBean {
+    /** Count of "beforeStart" invocations. */
+    public static volatile int beforeStartCnt;
+
+    /** Count of "afterStart" invocations. */
+    public static volatile int afterStartCnt;
+
+    /** {@inheritDoc} */
+    @Override public void onLifecycleEvent(LifecycleEventType evt) {
+        switch (evt) {
+            case BEFORE_NODE_START:
+                beforeStartCnt++;
+
+                break;
+
+            case AFTER_NODE_START:
+                afterStartCnt++;
+
+                break;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/java/org/apache/ignite/platform/lifecycle/PlatformJavaLifecycleTask.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/java/org/apache/ignite/platform/lifecycle/PlatformJavaLifecycleTask.java b/modules/platform/src/test/java/org/apache/ignite/platform/lifecycle/PlatformJavaLifecycleTask.java
new file mode 100644
index 0000000..ec01da1
--- /dev/null
+++ b/modules/platform/src/test/java/org/apache/ignite/platform/lifecycle/PlatformJavaLifecycleTask.java
@@ -0,0 +1,65 @@
+/*
+ * 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.platform.lifecycle;
+
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.compute.ComputeJob;
+import org.apache.ignite.compute.ComputeJobAdapter;
+import org.apache.ignite.compute.ComputeJobResult;
+import org.apache.ignite.compute.ComputeTaskAdapter;
+import org.jetbrains.annotations.Nullable;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Lifecycle task.
+ */
+public class PlatformJavaLifecycleTask extends ComputeTaskAdapter<Object, List<Integer>> {
+    /** {@inheritDoc} */
+    @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid,
+        @Nullable Object arg) {
+        Map<ComputeJob, ClusterNode> jobs = new HashMap<>();
+
+        jobs.put(new LifecycleJob(), subgrid.get(0));
+
+        return jobs;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public List<Integer> reduce(List<ComputeJobResult> results) {
+        return results.get(0).getData();
+    }
+
+    /**
+     * Job.
+     */
+    private static class LifecycleJob extends ComputeJobAdapter {
+        /** {@inheritDoc} */
+        @Nullable @Override public Object execute() {
+            List<Integer> res = new ArrayList<Integer>();
+
+            res.add(PlatformJavaLifecycleBean.beforeStartCnt);
+            res.add(PlatformJavaLifecycleBean.afterStartCnt);
+
+            return res;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index 69b0cd0..846211b 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -757,6 +757,12 @@
                                         <exclude>**/*.slnrel</exclude>
                                         <exclude>**/*.opensdf</exclude>
                                         <exclude>**/module.def</exclude>
+                                        <exclude>**/ignite-common.pc.in</exclude>
+                                        <exclude>**/*.csproj</exclude>
+                                        <exclude>**/*.fxcop</exclude>
+                                        <exclude>**/*.metaproj</exclude>
+                                        <exclude>**/*.metaproj.tmp</exclude>
+                                        <exclude>**/x64/Debug/**</exclude>
                                         <exclude>**/teamcity_boost.cpp</exclude>
                                         <exclude>**/teamcity_messages.h</exclude>
                                         <exclude>**/teamcity_messages.cpp</exclude>


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

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

Conflicts:
	pom.xml


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

Branch: refs/heads/ignite-843
Commit: ce1559bc8bf734837251e371c9caa9061c67c3a4
Parents: 5d3b129 e5eede5 2b16b57
Author: AKuznetsov <ak...@gridgain.com>
Authored: Sat Sep 5 09:31:16 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Sat Sep 5 09:31:16 2015 +0700

----------------------------------------------------------------------
 .../src/main/js/controllers/common-module.js    |  15 +--
 .../src/main/js/public/stylesheets/style.scss   |   5 +
 .../src/main/js/views/sql/sql.jade              |   3 +-
 .../platform/src/main/cpp/common/configure.ac   |   2 +-
 .../src/main/cpp/core-test/configure.ac         |   2 +-
 modules/platform/src/main/cpp/core/configure.ac |   2 +-
 .../Apache.Ignite.Core.csproj                   |   8 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 pom.xml                                         | 117 +++++++++++++++++++
 11 files changed, 145 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ce1559bc/pom.xml
----------------------------------------------------------------------
diff --cc pom.xml
index 8668495,8668495,6f9c848..f246996
--- a/pom.xml
+++ b/pom.xml
@@@@ -743,40 -743,40 -739,121 +743,157 @@@@
           </profile>
   
           <profile>
  +            <id>control-center</id>
  +            <modules>
  +                <module>modules/control-center-agent</module>
  +                <module>modules/control-center-web</module>
  +                <module>modules/schema-import-db</module>
  +            </modules>
  +        </profile>
  +
  +        <profile>
  +            <id>ignite-npm</id>
  +            <build>
  +                <plugins>
  +                    <plugin>
  +                        <groupId>org.codehaus.mojo</groupId>
  +                        <artifactId>exec-maven-plugin</artifactId>
  +                        <executions>
  +                            <execution>
  +                                <phase>compile</phase>
  +                                <goals>
  +                                    <goal>exec</goal>
  +                                </goals>
  +                            </execution>
  +                        </executions>
  +                        <configuration>
  +                            <executable>npm</executable>
  +                            <arguments>
  +                                <argument>publish</argument>
  +                                <argument>modules/nodejs/src/main/js</argument>
  +                            </arguments>
  +                        </configuration>
  +                    </plugin>
  +                </plugins>
  +            </build>
  +        </profile>
+++        
+++        <profile>
++             <id>update-versions</id>
++             <!-- updates dotnet & cpp versions -->
++             <build>
++                 <plugins>
++                     <plugin>
++                         <groupId>org.apache.maven.plugins</groupId>
++                         <artifactId>maven-antrun-plugin</artifactId>
++                         <version>1.7</version>
++                         <executions>
++                             <execution>
++                                 <id>update-versions</id>
++                                 <goals>
++                                     <goal>run</goal>
++                                 </goals>
++                                 <phase>validate</phase>
++                                 <configuration>
++                                     <target>
++                                         <script language="javascript">
++                                             function setClientVersion(ggVer, clientVer) {
++                                             var p = project.getProperty(ggVer);
++ 
++                                             if (java.util.regex.Pattern.matches(".*-p\\d+", p))
++                                             project.setProperty(clientVer, p.replace("-p", "."));
++                                             else
++                                             if (java.util.regex.Pattern.matches(".*-[a-zA-Z]+\\d+.*", p))
++                                             project.setProperty(clientVer, p.replaceAll("-[a-zA-Z]+(\\d+)?.*", ".$1"));
++                                             else
++                                             project.setProperty(clientVer, p);
++                                             }
++ 
++                                             function fix(dest, source) {
++                                             project.setProperty(dest, project.getProperty(source).replace("-SNAPSHOT",
++                                             ""));
++                                             }
++ 
++                                             fix('ignite.version.fixed', 'project.version');
++                                             fix('new.ignite.version.fixed', 'new.ignite.version');
++                                             setClientVersion('ignite.version.fixed', 'old.client.version');
++                                             setClientVersion('new.ignite.version.fixed', 'new.client.version');
++                                         </script>
++ 
++                                         <echo message="Update ignite.version in dotnet client" />
++                                         <echo message="${new.client.version}" />
++ 
++                                         <replaceregexp byline="true" encoding="UTF-8">
++                                             <regexp pattern="(\[assembly:\s*Assembly\w*Version\w*\(&quot;)\d.\d.\d(.\d)?(&quot;\)\])" />
++                                             <substitution expression="\1${new.client.version}\3" />
++                                             <fileset dir="${basedir}/">
++                                                 <include name="**/AssemblyInfo.cs" />
++                                                 <include name="**/AssemblyInfo.cpp" />
++                                             </fileset>
++                                         </replaceregexp>
++ 
++                                         <echo message="Update ignite.version in cpp client" />
++                                         <replaceregexp byline="true" encoding="UTF-8">
++                                             <regexp pattern="(AC_INIT.+\[)\d.\d.\d(.\d)?(\].+)" />
++                                             <substitution expression="\1${new.client.version}\3" />
++                                             <fileset dir="${basedir}/">
++                                                 <include name="**/configure.ac" />
++                                             </fileset>
++                                         </replaceregexp>
++ 
++                                         <replaceregexp byline="true" encoding="UTF-8">
++                                             <regexp pattern="(define GG_VERSION_STR_WIN &quot;)\d.\d.\d(.\d)?(&quot;)" />
++                                             <substitution expression="\1${new.client.version}\3" />
++                                             <fileset dir="${basedir}/">
++                                                 <include name="**/resource.h" />
++                                             </fileset>
++                                         </replaceregexp>
++ 
++                                         <replaceregexp byline="true" encoding="UTF-16">
++                                             <regexp pattern="(Version&quot;, &quot;)\d.\d.\d.\d(&quot;)" />
++                                             <substitution expression="\1${new.client.version}\2" />
++                                             <fileset dir="${basedir}/">
++                                                 <include name="**/Resource.rc" />
++                                             </fileset>
++                                         </replaceregexp>
++ 
++                                         <script language="javascript">
++                                             function setBinVersion(clientVer, binVer) {
++                                             var p = project.getProperty(clientVer).replace(".", ",");
++ 
++                                             if (p.split(',').length == 3)
++                                             project.setProperty(binVer, p + ',0');
++                                             else
++                                             project.setProperty(binVer, p);
++                                             }
++ 
++                                             setBinVersion('old.client.version', 'old.bin.version');
++                                             setBinVersion('new.client.version', 'new.bin.version');
++                                         </script>
++ 
++                                         <replaceregexp byline="true" encoding="UTF-8">
++                                             <regexp pattern="(define GG_VERSION_BIN_WIN )\d,\d,\d,\d" />
++                                             <substitution expression="\1${new.bin.version}" />
++                                             <fileset dir="${basedir}/">
++                                                 <include name="**/resource.h" />
++                                             </fileset>
++                                         </replaceregexp>
++ 
++                                         <replaceregexp byline="true" encoding="UTF-16">
++                                             <regexp pattern="(VERSION )\d,\d,\d,\d" />
++                                             <substitution expression="\1${new.bin.version}" />
++                                             <fileset dir="${basedir}/">
++                                                 <include name="**/Resource.rc" />
++                                             </fileset>
++                                         </replaceregexp>
++                                     </target>
++                                 </configuration>
++                             </execution>
++                         </executions>
++                     </plugin>
++                 </plugins>
++             </build>
++         </profile>
       </profiles>
   
       <build>


[05/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Portable/PortableApiSelfTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Portable/PortableApiSelfTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Portable/PortableApiSelfTest.cs
new file mode 100644
index 0000000..6f00fae
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Portable/PortableApiSelfTest.cs
@@ -0,0 +1,1787 @@
+/*
+ * 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 UnassignedField.Global
+// ReSharper disable CollectionNeverUpdated.Global
+namespace Apache.Ignite.Core.Tests.Portable
+{
+    using System;
+    using System.Collections;
+    using System.Collections.Generic;
+    using System.Linq;
+    using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Portable;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Portable builder self test.
+    /// </summary>
+    public class PortableApiSelfTest
+    {
+        /** Undefined type: Empty. */
+        private const string TypeEmpty = "EmptyUndefined";
+
+        /** Grid. */
+        private Ignite _grid;
+
+        /** Marshaller. */
+        private PortableMarshaller _marsh;
+
+        /// <summary>
+        /// Set up routine.
+        /// </summary>
+        [TestFixtureSetUp]
+        public void SetUp()
+        {
+            TestUtils.KillProcesses();
+
+            var cfg = new IgniteConfiguration
+            {
+                PortableConfiguration = new PortableConfiguration
+                {
+                    TypeConfigurations = new List<PortableTypeConfiguration>
+                    {
+                        new PortableTypeConfiguration(typeof (Empty)),
+                        new PortableTypeConfiguration(typeof (Primitives)),
+                        new PortableTypeConfiguration(typeof (PrimitiveArrays)),
+                        new PortableTypeConfiguration(typeof (StringDateGuidEnum)),
+                        new PortableTypeConfiguration(typeof (WithRaw)),
+                        new PortableTypeConfiguration(typeof (MetaOverwrite)),
+                        new PortableTypeConfiguration(typeof (NestedOuter)),
+                        new PortableTypeConfiguration(typeof (NestedInner)),
+                        new PortableTypeConfiguration(typeof (MigrationOuter)),
+                        new PortableTypeConfiguration(typeof (MigrationInner)),
+                        new PortableTypeConfiguration(typeof (InversionOuter)),
+                        new PortableTypeConfiguration(typeof (InversionInner)),
+                        new PortableTypeConfiguration(typeof (CompositeOuter)),
+                        new PortableTypeConfiguration(typeof (CompositeInner)),
+                        new PortableTypeConfiguration(typeof (CompositeArray)),
+                        new PortableTypeConfiguration(typeof (CompositeContainer)),
+                        new PortableTypeConfiguration(typeof (ToPortable)),
+                        new PortableTypeConfiguration(typeof (Remove)),
+                        new PortableTypeConfiguration(typeof (RemoveInner)),
+                        new PortableTypeConfiguration(typeof (BuilderInBuilderOuter)),
+                        new PortableTypeConfiguration(typeof (BuilderInBuilderInner)),
+                        new PortableTypeConfiguration(typeof (BuilderCollection)),
+                        new PortableTypeConfiguration(typeof (BuilderCollectionItem)),
+                        new PortableTypeConfiguration(typeof (DecimalHolder)),
+                        new PortableTypeConfiguration(TypeEmpty),
+                        TypeConfigurationNoMeta(typeof (EmptyNoMeta)),
+                        TypeConfigurationNoMeta(typeof (ToPortableNoMeta))
+                    },
+                    DefaultIdMapper = new IdMapper()
+                },
+                JvmClasspath = TestUtils.CreateTestClasspath(),
+                JvmOptions = new List<string>
+                {
+                    "-ea",
+                    "-Xcheck:jni",
+                    "-Xms4g",
+                    "-Xmx4g",
+                    "-DIGNITE_QUIET=false",
+                    "-Xnoagent",
+                    "-Djava.compiler=NONE",
+                    "-Xdebug",
+                    "-Xrunjdwp:transport=dt_socket,server=y,suspend=n,address=5005",
+                    "-XX:+HeapDumpOnOutOfMemoryError"
+                },
+                SpringConfigUrl = "config\\portable.xml"
+            };
+
+            _grid = (Ignite) Ignition.Start(cfg);
+
+            _marsh = _grid.Marshaller;
+        }
+
+        /// <summary>
+        /// Tear down routine.
+        /// </summary>
+        [TestFixtureTearDown]
+        public virtual void TearDown()
+        {
+            if (_grid != null)
+                Ignition.Stop(_grid.Name, true);
+
+            _grid = null;
+        }
+
+        /// <summary>
+        /// Ensure that portable engine is able to work with type names, which are not configured.
+        /// </summary>
+        [Test]
+        public void TestNonConfigured()
+        {
+            string typeName1 = "Type1";
+            string typeName2 = "Type2";
+            string field1 = "field1";
+            string field2 = "field2";
+
+            // 1. Ensure that builder works fine.
+            IPortableObject portObj1 = _grid.Portables().Builder(typeName1).SetField(field1, 1).Build();
+
+            Assert.AreEqual(typeName1, portObj1.Metadata().TypeName);
+            Assert.AreEqual(1, portObj1.Metadata().Fields.Count);
+            Assert.AreEqual(field1, portObj1.Metadata().Fields.First());
+            Assert.AreEqual(PortableTypeNames.TypeNameInt, portObj1.Metadata().FieldTypeName(field1));
+
+            Assert.AreEqual(1, portObj1.Field<int>(field1));
+
+            // 2. Ensure that object can be unmarshalled without deserialization.
+            byte[] data = ((PortableUserObject) portObj1).Data;
+
+            portObj1 = _grid.Marshaller.Unmarshal<IPortableObject>(data, PortableMode.ForcePortable);
+
+            Assert.AreEqual(typeName1, portObj1.Metadata().TypeName);
+            Assert.AreEqual(1, portObj1.Metadata().Fields.Count);
+            Assert.AreEqual(field1, portObj1.Metadata().Fields.First());
+            Assert.AreEqual(PortableTypeNames.TypeNameInt, portObj1.Metadata().FieldTypeName(field1));
+
+            Assert.AreEqual(1, portObj1.Field<int>(field1));
+
+            // 3. Ensure that we can nest one anonymous object inside another
+            IPortableObject portObj2 =
+                _grid.Portables().Builder(typeName2).SetField(field2, portObj1).Build();
+
+            Assert.AreEqual(typeName2, portObj2.Metadata().TypeName);
+            Assert.AreEqual(1, portObj2.Metadata().Fields.Count);
+            Assert.AreEqual(field2, portObj2.Metadata().Fields.First());
+            Assert.AreEqual(PortableTypeNames.TypeNameObject, portObj2.Metadata().FieldTypeName(field2));
+
+            portObj1 = portObj2.Field<IPortableObject>(field2);
+
+            Assert.AreEqual(typeName1, portObj1.Metadata().TypeName);
+            Assert.AreEqual(1, portObj1.Metadata().Fields.Count);
+            Assert.AreEqual(field1, portObj1.Metadata().Fields.First());
+            Assert.AreEqual(PortableTypeNames.TypeNameInt, portObj1.Metadata().FieldTypeName(field1));
+
+            Assert.AreEqual(1, portObj1.Field<int>(field1));
+
+            // 4. Ensure that we can unmarshal object with other nested object.
+            data = ((PortableUserObject) portObj2).Data;
+
+            portObj2 = _grid.Marshaller.Unmarshal<IPortableObject>(data, PortableMode.ForcePortable);
+
+            Assert.AreEqual(typeName2, portObj2.Metadata().TypeName);
+            Assert.AreEqual(1, portObj2.Metadata().Fields.Count);
+            Assert.AreEqual(field2, portObj2.Metadata().Fields.First());
+            Assert.AreEqual(PortableTypeNames.TypeNameObject, portObj2.Metadata().FieldTypeName(field2));
+
+            portObj1 = portObj2.Field<IPortableObject>(field2);
+
+            Assert.AreEqual(typeName1, portObj1.Metadata().TypeName);
+            Assert.AreEqual(1, portObj1.Metadata().Fields.Count);
+            Assert.AreEqual(field1, portObj1.Metadata().Fields.First());
+            Assert.AreEqual(PortableTypeNames.TypeNameInt, portObj1.Metadata().FieldTypeName(field1));
+
+            Assert.AreEqual(1, portObj1.Field<int>(field1));
+        }
+
+        /// <summary>
+        /// Test "ToPortable()" method.
+        /// </summary>
+        [Test]
+        public void TestToPortable()
+        {
+            DateTime date = DateTime.Now.ToUniversalTime();
+            Guid guid = Guid.NewGuid();
+
+            IPortables api = _grid.Portables();
+
+            // 1. Primitives.
+            Assert.AreEqual(1, api.ToPortable<byte>((byte)1));
+            Assert.AreEqual(1, api.ToPortable<short>((short)1));
+            Assert.AreEqual(1, api.ToPortable<int>(1));
+            Assert.AreEqual(1, api.ToPortable<long>((long)1));
+
+            Assert.AreEqual((float)1, api.ToPortable<float>((float)1));
+            Assert.AreEqual((double)1, api.ToPortable<double>((double)1));
+
+            Assert.AreEqual(true, api.ToPortable<bool>(true));
+            Assert.AreEqual('a', api.ToPortable<char>('a'));
+
+            // 2. Special types.
+            Assert.AreEqual("a", api.ToPortable<string>("a"));
+            Assert.AreEqual(date, api.ToPortable<DateTime>(date));
+            Assert.AreEqual(guid, api.ToPortable<Guid>(guid));
+            Assert.AreEqual(TestEnum.One, api.ToPortable<TestEnum>(TestEnum.One));
+
+            // 3. Arrays.
+            Assert.AreEqual(new byte[] { 1 }, api.ToPortable<byte[]>(new byte[] { 1 }));
+            Assert.AreEqual(new short[] { 1 }, api.ToPortable<short[]>(new short[] { 1 }));
+            Assert.AreEqual(new[] { 1 }, api.ToPortable<int[]>(new[] { 1 }));
+            Assert.AreEqual(new long[] { 1 }, api.ToPortable<long[]>(new long[] { 1 }));
+
+            Assert.AreEqual(new float[] { 1 }, api.ToPortable<float[]>(new float[] { 1 }));
+            Assert.AreEqual(new double[] { 1 }, api.ToPortable<double[]>(new double[] { 1 }));
+
+            Assert.AreEqual(new[] { true }, api.ToPortable<bool[]>(new[] { true }));
+            Assert.AreEqual(new[] { 'a' }, api.ToPortable<char[]>(new[] { 'a' }));
+
+            Assert.AreEqual(new[] { "a" }, api.ToPortable<string[]>(new[] { "a" }));
+            Assert.AreEqual(new[] { date }, api.ToPortable<DateTime[]>(new[] { date }));
+            Assert.AreEqual(new[] { guid }, api.ToPortable<Guid[]>(new[] { guid }));
+            Assert.AreEqual(new[] { TestEnum.One }, api.ToPortable<TestEnum[]>(new[] { TestEnum.One }));
+
+            // 4. Objects.
+            IPortableObject portObj = api.ToPortable<IPortableObject>(new ToPortable(1));
+
+            Assert.AreEqual(typeof(ToPortable).Name, portObj.Metadata().TypeName);
+            Assert.AreEqual(1, portObj.Metadata().Fields.Count);
+            Assert.AreEqual("Val", portObj.Metadata().Fields.First());
+            Assert.AreEqual(PortableTypeNames.TypeNameInt, portObj.Metadata().FieldTypeName("Val"));
+
+            Assert.AreEqual(1, portObj.Field<int>("val"));
+            Assert.AreEqual(1, portObj.Deserialize<ToPortable>().Val);
+
+            portObj = api.ToPortable<IPortableObject>(new ToPortableNoMeta(1));
+
+            Assert.AreEqual(0, portObj.Metadata().Fields.Count);
+
+            Assert.AreEqual(1, portObj.Field<int>("Val"));
+            Assert.AreEqual(1, portObj.Deserialize<ToPortableNoMeta>().Val);
+
+            // 5. Object array.
+            IPortableObject[] portObjArr = api.ToPortable<IPortableObject[]>(new[] { new ToPortable(1) });
+
+            Assert.AreEqual(1, portObjArr.Length);
+            Assert.AreEqual(1, portObjArr[0].Field<int>("Val"));
+            Assert.AreEqual(1, portObjArr[0].Deserialize<ToPortable>().Val);
+        }
+
+        /// <summary>
+        /// Test builder field remove logic.
+        /// </summary>
+        [Test]
+        public void TestRemove()
+        {
+            // Create empty object.
+            IPortableObject portObj = _grid.Portables().Builder(typeof(Remove)).Build();
+
+            Assert.IsNull(portObj.Field<object>("val"));
+            Assert.IsNull(portObj.Deserialize<Remove>().Val);
+
+            IPortableMetadata meta = portObj.Metadata();
+
+            Assert.AreEqual(typeof(Remove).Name, meta.TypeName);
+            Assert.AreEqual(0, meta.Fields.Count);
+
+            // Populate it with field.
+            IPortableBuilder builder = _grid.Portables().Builder(portObj);
+
+            Assert.IsNull(builder.GetField<object>("val"));
+
+            object val = 1;
+
+            builder.SetField("val", val);
+
+            Assert.AreEqual(val, builder.GetField<object>("val"));
+
+            portObj = builder.Build();
+
+            Assert.AreEqual(val, portObj.Field<object>("val"));
+            Assert.AreEqual(val, portObj.Deserialize<Remove>().Val);
+
+            meta = portObj.Metadata();
+
+            Assert.AreEqual(typeof(Remove).Name, meta.TypeName);
+            Assert.AreEqual(1, meta.Fields.Count);
+            Assert.AreEqual("val", meta.Fields.First());
+            Assert.AreEqual(PortableTypeNames.TypeNameObject, meta.FieldTypeName("val"));
+
+            // Perform field remove.
+            builder = _grid.Portables().Builder(portObj);
+
+            Assert.AreEqual(val, builder.GetField<object>("val"));
+
+            builder.RemoveField("val");
+            Assert.IsNull(builder.GetField<object>("val"));
+
+            builder.SetField("val", val);
+            Assert.AreEqual(val, builder.GetField<object>("val"));
+
+            builder.RemoveField("val");
+            Assert.IsNull(builder.GetField<object>("val"));
+
+            portObj = builder.Build();
+
+            Assert.IsNull(portObj.Field<object>("val"));
+            Assert.IsNull(portObj.Deserialize<Remove>().Val);
+
+            // Test correct removal of field being referenced by handle somewhere else.
+            RemoveInner inner = new RemoveInner(2);
+
+            portObj = _grid.Portables().Builder(typeof(Remove))
+                .SetField("val", inner)
+                .SetField("val2", inner)
+                .Build();
+
+            portObj = _grid.Portables().Builder(portObj).RemoveField("val").Build();
+
+            Remove obj = portObj.Deserialize<Remove>();
+
+            Assert.IsNull(obj.Val);
+            Assert.AreEqual(2, obj.Val2.Val);
+        }
+
+        /// <summary>
+        /// Test builder-in-builder scenario.
+        /// </summary>
+        [Test]
+        public void TestBuilderInBuilder()
+        {
+            // Test different builders assembly.
+            IPortableBuilder builderOuter = _grid.Portables().Builder(typeof(BuilderInBuilderOuter));
+            IPortableBuilder builderInner = _grid.Portables().Builder(typeof(BuilderInBuilderInner));
+
+            builderOuter.SetField<object>("inner", builderInner);
+            builderInner.SetField<object>("outer", builderOuter);
+
+            IPortableObject outerPortObj = builderOuter.Build();
+
+            IPortableMetadata meta = outerPortObj.Metadata();
+
+            Assert.AreEqual(typeof(BuilderInBuilderOuter).Name, meta.TypeName);
+            Assert.AreEqual(1, meta.Fields.Count);
+            Assert.AreEqual("inner", meta.Fields.First());
+            Assert.AreEqual(PortableTypeNames.TypeNameObject, meta.FieldTypeName("inner"));
+
+            IPortableObject innerPortObj = outerPortObj.Field<IPortableObject>("inner");
+
+            meta = innerPortObj.Metadata();
+
+            Assert.AreEqual(typeof(BuilderInBuilderInner).Name, meta.TypeName);
+            Assert.AreEqual(1, meta.Fields.Count);
+            Assert.AreEqual("outer", meta.Fields.First());
+            Assert.AreEqual(PortableTypeNames.TypeNameObject, meta.FieldTypeName("outer"));
+
+            BuilderInBuilderOuter outer = outerPortObj.Deserialize<BuilderInBuilderOuter>();
+
+            Assert.AreSame(outer, outer.Inner.Outer);
+
+            // Test same builders assembly.
+            innerPortObj = _grid.Portables().Builder(typeof(BuilderInBuilderInner)).Build();
+
+            outerPortObj = _grid.Portables().Builder(typeof(BuilderInBuilderOuter))
+                .SetField("inner", innerPortObj)
+                .SetField("inner2", innerPortObj)
+                .Build();
+
+            meta = outerPortObj.Metadata();
+
+            Assert.AreEqual(typeof(BuilderInBuilderOuter).Name, meta.TypeName);
+            Assert.AreEqual(2, meta.Fields.Count);
+            Assert.IsTrue(meta.Fields.Contains("inner"));
+            Assert.IsTrue(meta.Fields.Contains("inner2"));
+            Assert.AreEqual(PortableTypeNames.TypeNameObject, meta.FieldTypeName("inner"));
+            Assert.AreEqual(PortableTypeNames.TypeNameObject, meta.FieldTypeName("inner2"));
+
+            outer = outerPortObj.Deserialize<BuilderInBuilderOuter>();
+
+            Assert.AreSame(outer.Inner, outer.Inner2);
+
+            builderOuter = _grid.Portables().Builder(outerPortObj);
+            IPortableBuilder builderInner2 = builderOuter.GetField<IPortableBuilder>("inner2");
+
+            builderInner2.SetField("outer", builderOuter);
+
+            outerPortObj = builderOuter.Build();
+
+            outer = outerPortObj.Deserialize<BuilderInBuilderOuter>();
+
+            Assert.AreSame(outer, outer.Inner.Outer);
+            Assert.AreSame(outer.Inner, outer.Inner2);
+        }
+
+        /// <summary>
+        /// Test for decimals building.
+        /// </summary>
+        [Test]
+        public void TestDecimals()
+        {
+            IPortableObject portObj = _grid.Portables().Builder(typeof(DecimalHolder))
+                .SetField("val", decimal.One)
+                .SetField("valArr", new[] { decimal.MinusOne })
+                .Build();
+
+            IPortableMetadata meta = portObj.Metadata();
+
+            Assert.AreEqual(typeof(DecimalHolder).Name, meta.TypeName);
+            Assert.AreEqual(2, meta.Fields.Count);
+            Assert.IsTrue(meta.Fields.Contains("val"));
+            Assert.IsTrue(meta.Fields.Contains("valArr"));
+            Assert.AreEqual(PortableTypeNames.TypeNameDecimal, meta.FieldTypeName("val"));
+            Assert.AreEqual(PortableTypeNames.TypeNameArrayDecimal, meta.FieldTypeName("valArr"));
+
+            Assert.AreEqual(decimal.One, portObj.Field<decimal>("val"));
+            Assert.AreEqual(new[] { decimal.MinusOne }, portObj.Field<decimal[]>("valArr"));
+
+            DecimalHolder obj = portObj.Deserialize<DecimalHolder>();
+
+            Assert.AreEqual(decimal.One, obj.Val);
+            Assert.AreEqual(new[] { decimal.MinusOne }, obj.ValArr);
+        }
+
+        /// <summary>
+        /// Test for an object returning collection of builders.
+        /// </summary>
+        [Test]
+        public void TestBuilderCollection()
+        {
+            // Test collection with single element.
+            IPortableBuilder builderCol = _grid.Portables().Builder(typeof(BuilderCollection));
+            IPortableBuilder builderItem =
+                _grid.Portables().Builder(typeof(BuilderCollectionItem)).SetField("val", 1);
+
+            builderCol.SetField<ICollection>("col", new List<IPortableBuilder> { builderItem });
+
+            IPortableObject portCol = builderCol.Build();
+
+            IPortableMetadata meta = portCol.Metadata();
+
+            Assert.AreEqual(typeof(BuilderCollection).Name, meta.TypeName);
+            Assert.AreEqual(1, meta.Fields.Count);
+            Assert.AreEqual("col", meta.Fields.First());
+            Assert.AreEqual(PortableTypeNames.TypeNameCollection, meta.FieldTypeName("col"));
+
+            ICollection<IPortableObject> portColItems = portCol.Field<ICollection<IPortableObject>>("col");
+
+            Assert.AreEqual(1, portColItems.Count);
+
+            IPortableObject portItem = portColItems.First();
+
+            meta = portItem.Metadata();
+
+            Assert.AreEqual(typeof(BuilderCollectionItem).Name, meta.TypeName);
+            Assert.AreEqual(1, meta.Fields.Count);
+            Assert.AreEqual("val", meta.Fields.First());
+            Assert.AreEqual(PortableTypeNames.TypeNameInt, meta.FieldTypeName("val"));
+
+            BuilderCollection col = portCol.Deserialize<BuilderCollection>();
+
+            Assert.IsNotNull(col.Col);
+            Assert.AreEqual(1, col.Col.Count);
+            Assert.AreEqual(1, col.Col.First().Val);
+
+            // Add more portable objects to collection.
+            builderCol = _grid.Portables().Builder(portCol);
+
+            IList builderColItems = builderCol.GetField<IList>("col");
+
+            Assert.AreEqual(1, builderColItems.Count);
+
+            PortableBuilderImpl builderColItem = (PortableBuilderImpl) builderColItems[0];
+
+            builderColItem.SetField("val", 2); // Change nested value.
+
+            builderColItems.Add(builderColItem); // Add the same object to check handles.
+            builderColItems.Add(builderItem); // Add item from another builder.
+            builderColItems.Add(portItem); // Add item in portable form.
+
+            portCol = builderCol.Build();
+
+            col = portCol.Deserialize<BuilderCollection>();
+
+            Assert.AreEqual(4, col.Col.Count);
+
+            BuilderCollectionItem item0 = col.Col.ElementAt(0);
+            BuilderCollectionItem item1 = col.Col.ElementAt(1);
+            BuilderCollectionItem item2 = col.Col.ElementAt(2);
+            BuilderCollectionItem item3 = col.Col.ElementAt(3);
+
+            Assert.AreEqual(2, item0.Val);
+
+            Assert.AreSame(item0, item1);
+            Assert.AreNotSame(item0, item2);
+            Assert.AreNotSame(item0, item3);
+
+            Assert.AreEqual(1, item2.Val);
+            Assert.AreEqual(1, item3.Val);
+
+            Assert.AreNotSame(item2, item3);
+
+            // Test handle update inside collection.
+            builderCol = _grid.Portables().Builder(portCol);
+
+            builderColItems = builderCol.GetField<IList>("col");
+
+            ((PortableBuilderImpl) builderColItems[1]).SetField("val", 3);
+
+            portCol = builderCol.Build();
+
+            col = portCol.Deserialize<BuilderCollection>();
+
+            item0 = col.Col.ElementAt(0);
+            item1 = col.Col.ElementAt(1);
+
+            Assert.AreEqual(3, item0.Val);
+            Assert.AreSame(item0, item1);
+        }
+
+        /// <summary>
+        /// Test build of an empty object.
+        /// </summary>
+        [Test]
+        public void TestEmptyDefined()
+        {
+            IPortableObject portObj = _grid.Portables().Builder(typeof(Empty)).Build();
+
+            Assert.IsNotNull(portObj);
+            Assert.AreEqual(0, portObj.GetHashCode());
+
+            IPortableMetadata meta = portObj.Metadata();
+
+            Assert.IsNotNull(meta);
+            Assert.AreEqual(typeof(Empty).Name, meta.TypeName);
+            Assert.AreEqual(0, meta.Fields.Count);
+
+            Empty obj = portObj.Deserialize<Empty>();
+
+            Assert.IsNotNull(obj);
+        }
+
+        /// <summary>
+        /// Test build of an empty object with disabled metadata.
+        /// </summary>
+        [Test]
+        public void TestEmptyNoMeta()
+        {
+            IPortableObject portObj = _grid.Portables().Builder(typeof(EmptyNoMeta)).Build();
+
+            Assert.IsNotNull(portObj);
+            Assert.AreEqual(0, portObj.GetHashCode());
+
+            EmptyNoMeta obj = portObj.Deserialize<EmptyNoMeta>();
+
+            Assert.IsNotNull(obj);
+        }
+
+        /// <summary>
+        /// Test build of an empty undefined object.
+        /// </summary>
+        [Test]
+        public void TestEmptyUndefined()
+        {
+            IPortableObject portObj = _grid.Portables().Builder(TypeEmpty).Build();
+
+            Assert.IsNotNull(portObj);
+            Assert.AreEqual(0, portObj.GetHashCode());
+
+            IPortableMetadata meta = portObj.Metadata();
+
+            Assert.IsNotNull(meta);
+            Assert.AreEqual(TypeEmpty, meta.TypeName);
+            Assert.AreEqual(0, meta.Fields.Count);
+        }
+
+        /// <summary>
+        /// Test object rebuild with no changes.
+        /// </summary>
+        [Test]
+        public void TestEmptyRebuild()
+        {
+            var portObj = (PortableUserObject) _grid.Portables().Builder(typeof(EmptyNoMeta)).Build();
+
+            PortableUserObject newPortObj = (PortableUserObject) _grid.Portables().Builder(portObj).Build();
+
+            Assert.AreEqual(portObj.Data, newPortObj.Data);
+        }
+
+        /// <summary>
+        /// Test hash code alteration.
+        /// </summary>
+        [Test]
+        public void TestHashCodeChange()
+        {
+            IPortableObject portObj = _grid.Portables().Builder(typeof(EmptyNoMeta)).HashCode(100).Build();
+
+            Assert.AreEqual(100, portObj.GetHashCode());
+        }
+
+        /// <summary>
+        /// Test primitive fields setting.
+        /// </summary>
+        [Test]
+        public void TestPrimitiveFields()
+        {
+            IPortableObject portObj = _grid.Portables().Builder(typeof(Primitives))
+                .SetField<byte>("fByte", 1)
+                .SetField("fBool", true)
+                .SetField<short>("fShort", 2)
+                .SetField("fChar", 'a')
+                .SetField("fInt", 3)
+                .SetField<long>("fLong", 4)
+                .SetField<float>("fFloat", 5)
+                .SetField<double>("fDouble", 6)
+                .HashCode(100)
+                .Build();
+
+            Assert.AreEqual(100, portObj.GetHashCode());
+
+            IPortableMetadata meta = portObj.Metadata();
+
+            Assert.AreEqual(typeof(Primitives).Name, meta.TypeName);
+
+            Assert.AreEqual(8, meta.Fields.Count);
+
+            Assert.AreEqual(PortableTypeNames.TypeNameByte, meta.FieldTypeName("fByte"));
+            Assert.AreEqual(PortableTypeNames.TypeNameBool, meta.FieldTypeName("fBool"));
+            Assert.AreEqual(PortableTypeNames.TypeNameShort, meta.FieldTypeName("fShort"));
+            Assert.AreEqual(PortableTypeNames.TypeNameChar, meta.FieldTypeName("fChar"));
+            Assert.AreEqual(PortableTypeNames.TypeNameInt, meta.FieldTypeName("fInt"));
+            Assert.AreEqual(PortableTypeNames.TypeNameLong, meta.FieldTypeName("fLong"));
+            Assert.AreEqual(PortableTypeNames.TypeNameFloat, meta.FieldTypeName("fFloat"));
+            Assert.AreEqual(PortableTypeNames.TypeNameDouble, meta.FieldTypeName("fDouble"));
+
+            Assert.AreEqual(1, portObj.Field<byte>("fByte"));
+            Assert.AreEqual(true, portObj.Field<bool>("fBool"));
+            Assert.AreEqual(2, portObj.Field<short>("fShort"));
+            Assert.AreEqual('a', portObj.Field<char>("fChar"));
+            Assert.AreEqual(3, portObj.Field<int>("fInt"));
+            Assert.AreEqual(4, portObj.Field<long>("fLong"));
+            Assert.AreEqual(5, portObj.Field<float>("fFloat"));
+            Assert.AreEqual(6, portObj.Field<double>("fDouble"));
+
+            Primitives obj = portObj.Deserialize<Primitives>();
+
+            Assert.AreEqual(1, obj.FByte);
+            Assert.AreEqual(true, obj.FBool);
+            Assert.AreEqual(2, obj.FShort);
+            Assert.AreEqual('a', obj.FChar);
+            Assert.AreEqual(3, obj.FInt);
+            Assert.AreEqual(4, obj.FLong);
+            Assert.AreEqual(5, obj.FFloat);
+            Assert.AreEqual(6, obj.FDouble);
+
+            // Overwrite.
+            portObj = _grid.Portables().Builder(portObj)
+                .SetField<byte>("fByte", 7)
+                .SetField("fBool", false)
+                .SetField<short>("fShort", 8)
+                .SetField("fChar", 'b')
+                .SetField("fInt", 9)
+                .SetField<long>("fLong", 10)
+                .SetField<float>("fFloat", 11)
+                .SetField<double>("fDouble", 12)
+                .HashCode(200)
+                .Build();
+
+            Assert.AreEqual(200, portObj.GetHashCode());
+
+            Assert.AreEqual(7, portObj.Field<byte>("fByte"));
+            Assert.AreEqual(false, portObj.Field<bool>("fBool"));
+            Assert.AreEqual(8, portObj.Field<short>("fShort"));
+            Assert.AreEqual('b', portObj.Field<char>("fChar"));
+            Assert.AreEqual(9, portObj.Field<int>("fInt"));
+            Assert.AreEqual(10, portObj.Field<long>("fLong"));
+            Assert.AreEqual(11, portObj.Field<float>("fFloat"));
+            Assert.AreEqual(12, portObj.Field<double>("fDouble"));
+
+            obj = portObj.Deserialize<Primitives>();
+
+            Assert.AreEqual(7, obj.FByte);
+            Assert.AreEqual(false, obj.FBool);
+            Assert.AreEqual(8, obj.FShort);
+            Assert.AreEqual('b', obj.FChar);
+            Assert.AreEqual(9, obj.FInt);
+            Assert.AreEqual(10, obj.FLong);
+            Assert.AreEqual(11, obj.FFloat);
+            Assert.AreEqual(12, obj.FDouble);
+        }
+
+        /// <summary>
+        /// Test primitive array fields setting.
+        /// </summary>
+        [Test]
+        public void TestPrimitiveArrayFields()
+        {
+            IPortableObject portObj = _grid.Portables().Builder(typeof(PrimitiveArrays))
+                .SetField("fByte", new byte[] { 1 })
+                .SetField("fBool", new[] { true })
+                .SetField("fShort", new short[] { 2 })
+                .SetField("fChar", new[] { 'a' })
+                .SetField("fInt", new[] { 3 })
+                .SetField("fLong", new long[] { 4 })
+                .SetField("fFloat", new float[] { 5 })
+                .SetField("fDouble", new double[] { 6 })
+                .HashCode(100)
+                .Build();
+
+            Assert.AreEqual(100, portObj.GetHashCode());
+
+            IPortableMetadata meta = portObj.Metadata();
+
+            Assert.AreEqual(typeof(PrimitiveArrays).Name, meta.TypeName);
+
+            Assert.AreEqual(8, meta.Fields.Count);
+
+            Assert.AreEqual(PortableTypeNames.TypeNameArrayByte, meta.FieldTypeName("fByte"));
+            Assert.AreEqual(PortableTypeNames.TypeNameArrayBool, meta.FieldTypeName("fBool"));
+            Assert.AreEqual(PortableTypeNames.TypeNameArrayShort, meta.FieldTypeName("fShort"));
+            Assert.AreEqual(PortableTypeNames.TypeNameArrayChar, meta.FieldTypeName("fChar"));
+            Assert.AreEqual(PortableTypeNames.TypeNameArrayInt, meta.FieldTypeName("fInt"));
+            Assert.AreEqual(PortableTypeNames.TypeNameArrayLong, meta.FieldTypeName("fLong"));
+            Assert.AreEqual(PortableTypeNames.TypeNameArrayFloat, meta.FieldTypeName("fFloat"));
+            Assert.AreEqual(PortableTypeNames.TypeNameArrayDouble, meta.FieldTypeName("fDouble"));
+
+            Assert.AreEqual(new byte[] { 1 }, portObj.Field<byte[]>("fByte"));
+            Assert.AreEqual(new[] { true }, portObj.Field<bool[]>("fBool"));
+            Assert.AreEqual(new short[] { 2 }, portObj.Field<short[]>("fShort"));
+            Assert.AreEqual(new[] { 'a' }, portObj.Field<char[]>("fChar"));
+            Assert.AreEqual(new[] { 3 }, portObj.Field<int[]>("fInt"));
+            Assert.AreEqual(new long[] { 4 }, portObj.Field<long[]>("fLong"));
+            Assert.AreEqual(new float[] { 5 }, portObj.Field<float[]>("fFloat"));
+            Assert.AreEqual(new double[] { 6 }, portObj.Field<double[]>("fDouble"));
+
+            PrimitiveArrays obj = portObj.Deserialize<PrimitiveArrays>();
+
+            Assert.AreEqual(new byte[] { 1 }, obj.FByte);
+            Assert.AreEqual(new[] { true }, obj.FBool);
+            Assert.AreEqual(new short[] { 2 }, obj.FShort);
+            Assert.AreEqual(new[] { 'a' }, obj.FChar);
+            Assert.AreEqual(new[] { 3 }, obj.FInt);
+            Assert.AreEqual(new long[] { 4 }, obj.FLong);
+            Assert.AreEqual(new float[] { 5 }, obj.FFloat);
+            Assert.AreEqual(new double[] { 6 }, obj.FDouble);
+
+            // Overwrite.
+            portObj = _grid.Portables().Builder(portObj)
+                .SetField("fByte", new byte[] { 7 })
+                .SetField("fBool", new[] { false })
+                .SetField("fShort", new short[] { 8 })
+                .SetField("fChar", new[] { 'b' })
+                .SetField("fInt", new[] { 9 })
+                .SetField("fLong", new long[] { 10 })
+                .SetField("fFloat", new float[] { 11 })
+                .SetField("fDouble", new double[] { 12 })
+                .HashCode(200)
+                .Build();
+
+            Assert.AreEqual(200, portObj.GetHashCode());
+
+            Assert.AreEqual(new byte[] { 7 }, portObj.Field<byte[]>("fByte"));
+            Assert.AreEqual(new[] { false }, portObj.Field<bool[]>("fBool"));
+            Assert.AreEqual(new short[] { 8 }, portObj.Field<short[]>("fShort"));
+            Assert.AreEqual(new[] { 'b' }, portObj.Field<char[]>("fChar"));
+            Assert.AreEqual(new[] { 9 }, portObj.Field<int[]>("fInt"));
+            Assert.AreEqual(new long[] { 10 }, portObj.Field<long[]>("fLong"));
+            Assert.AreEqual(new float[] { 11 }, portObj.Field<float[]>("fFloat"));
+            Assert.AreEqual(new double[] { 12 }, portObj.Field<double[]>("fDouble"));
+
+            obj = portObj.Deserialize<PrimitiveArrays>();
+
+            Assert.AreEqual(new byte[] { 7 }, obj.FByte);
+            Assert.AreEqual(new[] { false }, obj.FBool);
+            Assert.AreEqual(new short[] { 8 }, obj.FShort);
+            Assert.AreEqual(new[] { 'b' }, obj.FChar);
+            Assert.AreEqual(new[] { 9 }, obj.FInt);
+            Assert.AreEqual(new long[] { 10 }, obj.FLong);
+            Assert.AreEqual(new float[] { 11 }, obj.FFloat);
+            Assert.AreEqual(new double[] { 12 }, obj.FDouble);
+        }
+
+        /// <summary>
+        /// Test non-primitive fields and their array counterparts.
+        /// </summary>
+        [Test]
+        public void TestStringDateGuidEnum()
+        {
+            DateTime date = DateTime.Now.ToUniversalTime();
+            DateTime? nDate = DateTime.Now.ToUniversalTime();
+
+            Guid guid = Guid.NewGuid();
+            Guid? nGuid = Guid.NewGuid();
+
+            IPortableObject portObj = _grid.Portables().Builder(typeof(StringDateGuidEnum))
+                .SetField("fStr", "str")
+                .SetField("fDate", date)
+                .SetField("fNDate", nDate)
+                .SetField("fGuid", guid)
+                .SetField("fNGuid", nGuid)
+                .SetField("fEnum", TestEnum.One)
+                .SetField("fStrArr", new[] { "str" })
+                .SetField("fDateArr", new[] { nDate })
+                .SetField("fGuidArr", new[] { nGuid })
+                .SetField("fEnumArr", new[] { TestEnum.One })
+                .HashCode(100)
+                .Build();
+
+            Assert.AreEqual(100, portObj.GetHashCode());
+
+            IPortableMetadata meta = portObj.Metadata();
+
+            Assert.AreEqual(typeof(StringDateGuidEnum).Name, meta.TypeName);
+
+            Assert.AreEqual(10, meta.Fields.Count);
+
+            Assert.AreEqual(PortableTypeNames.TypeNameString, meta.FieldTypeName("fStr"));
+            Assert.AreEqual(PortableTypeNames.TypeNameDate, meta.FieldTypeName("fDate"));
+            Assert.AreEqual(PortableTypeNames.TypeNameDate, meta.FieldTypeName("fNDate"));
+            Assert.AreEqual(PortableTypeNames.TypeNameGuid, meta.FieldTypeName("fGuid"));
+            Assert.AreEqual(PortableTypeNames.TypeNameGuid, meta.FieldTypeName("fNGuid"));
+            Assert.AreEqual(PortableTypeNames.TypeNameEnum, meta.FieldTypeName("fEnum"));
+            Assert.AreEqual(PortableTypeNames.TypeNameArrayString, meta.FieldTypeName("fStrArr"));
+            Assert.AreEqual(PortableTypeNames.TypeNameArrayDate, meta.FieldTypeName("fDateArr"));
+            Assert.AreEqual(PortableTypeNames.TypeNameArrayGuid, meta.FieldTypeName("fGuidArr"));
+            Assert.AreEqual(PortableTypeNames.TypeNameArrayEnum, meta.FieldTypeName("fEnumArr"));
+
+            Assert.AreEqual("str", portObj.Field<string>("fStr"));
+            Assert.AreEqual(date, portObj.Field<DateTime>("fDate"));
+            Assert.AreEqual(nDate, portObj.Field<DateTime?>("fNDate"));
+            Assert.AreEqual(guid, portObj.Field<Guid>("fGuid"));
+            Assert.AreEqual(nGuid, portObj.Field<Guid?>("fNGuid"));
+            Assert.AreEqual(TestEnum.One, portObj.Field<TestEnum>("fEnum"));
+            Assert.AreEqual(new[] { "str" }, portObj.Field<string[]>("fStrArr"));
+            Assert.AreEqual(new[] { nDate }, portObj.Field<DateTime?[]>("fDateArr"));
+            Assert.AreEqual(new[] { nGuid }, portObj.Field<Guid?[]>("fGuidArr"));
+            Assert.AreEqual(new[] { TestEnum.One }, portObj.Field<TestEnum[]>("fEnumArr"));
+
+            StringDateGuidEnum obj = portObj.Deserialize<StringDateGuidEnum>();
+
+            Assert.AreEqual("str", obj.FStr);
+            Assert.AreEqual(date, obj.FDate);
+            Assert.AreEqual(nDate, obj.FnDate);
+            Assert.AreEqual(guid, obj.FGuid);
+            Assert.AreEqual(nGuid, obj.FnGuid);
+            Assert.AreEqual(TestEnum.One, obj.FEnum);
+            Assert.AreEqual(new[] { "str" }, obj.FStrArr);
+            Assert.AreEqual(new[] { nDate }, obj.FDateArr);
+            Assert.AreEqual(new[] { nGuid }, obj.FGuidArr);
+            Assert.AreEqual(new[] { TestEnum.One }, obj.FEnumArr);
+
+            // Overwrite.
+            date = DateTime.Now.ToUniversalTime();
+            nDate = DateTime.Now.ToUniversalTime();
+
+            guid = Guid.NewGuid();
+            nGuid = Guid.NewGuid();
+
+            portObj = _grid.Portables().Builder(typeof(StringDateGuidEnum))
+                .SetField("fStr", "str2")
+                .SetField("fDate", date)
+                .SetField("fNDate", nDate)
+                .SetField("fGuid", guid)
+                .SetField("fNGuid", nGuid)
+                .SetField("fEnum", TestEnum.Two)
+                .SetField("fStrArr", new[] { "str2" })
+                .SetField("fDateArr", new[] { nDate })
+                .SetField("fGuidArr", new[] { nGuid })
+                .SetField("fEnumArr", new[] { TestEnum.Two })
+                .HashCode(200)
+                .Build();
+
+            Assert.AreEqual(200, portObj.GetHashCode());
+
+            Assert.AreEqual("str2", portObj.Field<string>("fStr"));
+            Assert.AreEqual(date, portObj.Field<DateTime>("fDate"));
+            Assert.AreEqual(nDate, portObj.Field<DateTime?>("fNDate"));
+            Assert.AreEqual(guid, portObj.Field<Guid>("fGuid"));
+            Assert.AreEqual(nGuid, portObj.Field<Guid?>("fNGuid"));
+            Assert.AreEqual(TestEnum.Two, portObj.Field<TestEnum>("fEnum"));
+            Assert.AreEqual(new[] { "str2" }, portObj.Field<string[]>("fStrArr"));
+            Assert.AreEqual(new[] { nDate }, portObj.Field<DateTime?[]>("fDateArr"));
+            Assert.AreEqual(new[] { nGuid }, portObj.Field<Guid?[]>("fGuidArr"));
+            Assert.AreEqual(new[] { TestEnum.Two }, portObj.Field<TestEnum[]>("fEnumArr"));
+
+            obj = portObj.Deserialize<StringDateGuidEnum>();
+
+            Assert.AreEqual("str2", obj.FStr);
+            Assert.AreEqual(date, obj.FDate);
+            Assert.AreEqual(nDate, obj.FnDate);
+            Assert.AreEqual(guid, obj.FGuid);
+            Assert.AreEqual(nGuid, obj.FnGuid);
+            Assert.AreEqual(TestEnum.Two, obj.FEnum);
+            Assert.AreEqual(new[] { "str2" }, obj.FStrArr);
+            Assert.AreEqual(new[] { nDate }, obj.FDateArr);
+            Assert.AreEqual(new[] { nGuid }, obj.FGuidArr);
+            Assert.AreEqual(new[] { TestEnum.Two }, obj.FEnumArr);
+        }
+
+        /// <summary>
+        /// Test arrays.
+        /// </summary>
+        [Test]
+        public void TestCompositeArray()
+        {
+            // 1. Test simple array.
+            CompositeInner[] inArr = { new CompositeInner(1) };
+
+            IPortableObject portObj = _grid.Portables().Builder(typeof(CompositeArray)).HashCode(100)
+                .SetField("inArr", inArr).Build();
+
+            IPortableMetadata meta = portObj.Metadata();
+
+            Assert.AreEqual(typeof(CompositeArray).Name, meta.TypeName);
+            Assert.AreEqual(1, meta.Fields.Count);
+            Assert.AreEqual(PortableTypeNames.TypeNameArrayObject, meta.FieldTypeName("inArr"));
+
+            Assert.AreEqual(100, portObj.GetHashCode());
+
+            IPortableObject[] portInArr = portObj.Field<IPortableObject[]>("inArr");
+
+            Assert.AreEqual(1, portInArr.Length);
+            Assert.AreEqual(1, portInArr[0].Field<int>("val"));
+
+            CompositeArray arr = portObj.Deserialize<CompositeArray>();
+
+            Assert.IsNull(arr.OutArr);
+            Assert.AreEqual(1, arr.InArr.Length);
+            Assert.AreEqual(1, arr.InArr[0].Val);
+
+            // 2. Test addition to array.
+            portInArr = new[] { portInArr[0], null };
+
+            portObj = _grid.Portables().Builder(portObj).HashCode(200)
+                .SetField("inArr", portInArr).Build();
+
+            Assert.AreEqual(200, portObj.GetHashCode());
+
+            portInArr = portObj.Field<IPortableObject[]>("inArr");
+
+            Assert.AreEqual(2, portInArr.Length);
+            Assert.AreEqual(1, portInArr[0].Field<int>("val"));
+            Assert.IsNull(portInArr[1]);
+
+            arr = portObj.Deserialize<CompositeArray>();
+
+            Assert.IsNull(arr.OutArr);
+            Assert.AreEqual(2, arr.InArr.Length);
+            Assert.AreEqual(1, arr.InArr[0].Val);
+            Assert.IsNull(arr.InArr[1]);
+
+            portInArr[1] = _grid.Portables().Builder(typeof(CompositeInner)).SetField("val", 2).Build();
+
+            portObj = _grid.Portables().Builder(portObj).HashCode(300)
+                .SetField("inArr", portInArr).Build();
+
+            Assert.AreEqual(300, portObj.GetHashCode());
+
+            portInArr = portObj.Field<IPortableObject[]>("inArr");
+
+            Assert.AreEqual(2, portInArr.Length);
+            Assert.AreEqual(1, portInArr[0].Field<int>("val"));
+            Assert.AreEqual(2, portInArr[1].Field<int>("val"));
+
+            arr = portObj.Deserialize<CompositeArray>();
+
+            Assert.IsNull(arr.OutArr);
+            Assert.AreEqual(2, arr.InArr.Length);
+            Assert.AreEqual(1, arr.InArr[0].Val);
+            Assert.AreEqual(2, arr.InArr[1].Val);
+
+            // 3. Test top-level handle inversion.
+            CompositeInner inner = new CompositeInner(1);
+
+            inArr = new[] { inner, inner };
+
+            portObj = _grid.Portables().Builder(typeof(CompositeArray)).HashCode(100)
+                .SetField("inArr", inArr).Build();
+
+            Assert.AreEqual(100, portObj.GetHashCode());
+
+            portInArr = portObj.Field<IPortableObject[]>("inArr");
+
+            Assert.AreEqual(2, portInArr.Length);
+            Assert.AreEqual(1, portInArr[0].Field<int>("val"));
+            Assert.AreEqual(1, portInArr[1].Field<int>("val"));
+
+            arr = portObj.Deserialize<CompositeArray>();
+
+            Assert.IsNull(arr.OutArr);
+            Assert.AreEqual(2, arr.InArr.Length);
+            Assert.AreEqual(1, arr.InArr[0].Val);
+            Assert.AreEqual(1, arr.InArr[1].Val);
+
+            portInArr[0] = _grid.Portables().Builder(typeof(CompositeInner)).SetField("val", 2).Build();
+
+            portObj = _grid.Portables().Builder(portObj).HashCode(200)
+                .SetField("inArr", portInArr).Build();
+
+            Assert.AreEqual(200, portObj.GetHashCode());
+
+            portInArr = portObj.Field<IPortableObject[]>("inArr");
+
+            Assert.AreEqual(2, portInArr.Length);
+            Assert.AreEqual(2, portInArr[0].Field<int>("val"));
+            Assert.AreEqual(1, portInArr[1].Field<int>("val"));
+
+            arr = portObj.Deserialize<CompositeArray>();
+
+            Assert.IsNull(arr.OutArr);
+            Assert.AreEqual(2, arr.InArr.Length);
+            Assert.AreEqual(2, arr.InArr[0].Val);
+            Assert.AreEqual(1, arr.InArr[1].Val);
+
+            // 4. Test nested object handle inversion.
+            CompositeOuter[] outArr = { new CompositeOuter(inner), new CompositeOuter(inner) };
+
+            portObj = _grid.Portables().Builder(typeof(CompositeArray)).HashCode(100)
+                .SetField("outArr", outArr).Build();
+
+            meta = portObj.Metadata();
+
+            Assert.AreEqual(typeof(CompositeArray).Name, meta.TypeName);
+            Assert.AreEqual(2, meta.Fields.Count);
+            Assert.AreEqual(PortableTypeNames.TypeNameArrayObject, meta.FieldTypeName("inArr"));
+            Assert.AreEqual(PortableTypeNames.TypeNameArrayObject, meta.FieldTypeName("outArr"));
+
+            Assert.AreEqual(100, portObj.GetHashCode());
+
+            IPortableObject[] portOutArr = portObj.Field<IPortableObject[]>("outArr");
+
+            Assert.AreEqual(2, portOutArr.Length);
+            Assert.AreEqual(1, portOutArr[0].Field<IPortableObject>("inner").Field<int>("val"));
+            Assert.AreEqual(1, portOutArr[1].Field<IPortableObject>("inner").Field<int>("val"));
+
+            arr = portObj.Deserialize<CompositeArray>();
+
+            Assert.IsNull(arr.InArr);
+            Assert.AreEqual(2, arr.OutArr.Length);
+            Assert.AreEqual(1, arr.OutArr[0].Inner.Val);
+            Assert.AreEqual(1, arr.OutArr[1].Inner.Val);
+
+            portOutArr[0] = _grid.Portables().Builder(typeof(CompositeOuter))
+                .SetField("inner", new CompositeInner(2)).Build();
+
+            portObj = _grid.Portables().Builder(portObj).HashCode(200)
+                .SetField("outArr", portOutArr).Build();
+
+            Assert.AreEqual(200, portObj.GetHashCode());
+
+            portInArr = portObj.Field<IPortableObject[]>("outArr");
+
+            Assert.AreEqual(2, portInArr.Length);
+            Assert.AreEqual(2, portOutArr[0].Field<IPortableObject>("inner").Field<int>("val"));
+            Assert.AreEqual(1, portOutArr[1].Field<IPortableObject>("inner").Field<int>("val"));
+
+            arr = portObj.Deserialize<CompositeArray>();
+
+            Assert.IsNull(arr.InArr);
+            Assert.AreEqual(2, arr.OutArr.Length);
+            Assert.AreEqual(2, arr.OutArr[0].Inner.Val);
+            Assert.AreEqual(1, arr.OutArr[1].Inner.Val);
+        }
+
+        /// <summary>
+        /// Test container types other than array.
+        /// </summary>
+        [Test]
+        public void TestCompositeContainer()
+        {
+            ArrayList col = new ArrayList();
+            ICollection<CompositeInner> gCol = new List<CompositeInner>();
+            IDictionary dict = new Hashtable();
+            IDictionary<int, CompositeInner> gDict = new Dictionary<int, CompositeInner>();
+
+            col.Add(new CompositeInner(1));
+            gCol.Add(new CompositeInner(2));
+            dict[3] = new CompositeInner(3);
+            gDict[4] = new CompositeInner(4);
+
+            IPortableObject portObj = _grid.Portables().Builder(typeof(CompositeContainer)).HashCode(100)
+                .SetField<ICollection>("col", col)
+                .SetField("gCol", gCol)
+                .SetField("dict", dict)
+                .SetField("gDict", gDict).Build();
+
+            // 1. Check meta.
+            IPortableMetadata meta = portObj.Metadata();
+
+            Assert.AreEqual(typeof(CompositeContainer).Name, meta.TypeName);
+
+            Assert.AreEqual(4, meta.Fields.Count);
+            Assert.AreEqual(PortableTypeNames.TypeNameCollection, meta.FieldTypeName("col"));
+            Assert.AreEqual(PortableTypeNames.TypeNameCollection, meta.FieldTypeName("gCol"));
+            Assert.AreEqual(PortableTypeNames.TypeNameMap, meta.FieldTypeName("dict"));
+            Assert.AreEqual(PortableTypeNames.TypeNameMap, meta.FieldTypeName("gDict"));
+
+            // 2. Check in portable form.
+            Assert.AreEqual(1, portObj.Field<ICollection>("col").Count);
+            Assert.AreEqual(1, portObj.Field<ICollection>("col").OfType<IPortableObject>().First()
+                .Field<int>("val"));
+
+            Assert.AreEqual(1, portObj.Field<ICollection<IPortableObject>>("gCol").Count);
+            Assert.AreEqual(2, portObj.Field<ICollection<IPortableObject>>("gCol").First().Field<int>("val"));
+
+            Assert.AreEqual(1, portObj.Field<IDictionary>("dict").Count);
+            Assert.AreEqual(3, ((IPortableObject) portObj.Field<IDictionary>("dict")[3]).Field<int>("val"));
+
+            Assert.AreEqual(1, portObj.Field<IDictionary<int, IPortableObject>>("gDict").Count);
+            Assert.AreEqual(4, portObj.Field<IDictionary<int, IPortableObject>>("gDict")[4].Field<int>("val"));
+
+            // 3. Check in deserialized form.
+            CompositeContainer obj = portObj.Deserialize<CompositeContainer>();
+
+            Assert.AreEqual(1, obj.Col.Count);
+            Assert.AreEqual(1, obj.Col.OfType<CompositeInner>().First().Val);
+
+            Assert.AreEqual(1, obj.GCol.Count);
+            Assert.AreEqual(2, obj.GCol.First().Val);
+
+            Assert.AreEqual(1, obj.Dict.Count);
+            Assert.AreEqual(3, ((CompositeInner) obj.Dict[3]).Val);
+
+            Assert.AreEqual(1, obj.GDict.Count);
+            Assert.AreEqual(4, obj.GDict[4].Val);
+        }
+
+        /// <summary>
+        /// Ensure that raw data is not lost during build.
+        /// </summary>
+        [Test]
+        public void TestRawData()
+        {
+            var raw = new WithRaw
+            {
+                A = 1,
+                B = 2
+            };
+
+            var portObj = _marsh.Unmarshal<IPortableObject>(_marsh.Marshal(raw), PortableMode.ForcePortable);
+
+            raw = portObj.Deserialize<WithRaw>();
+
+            Assert.AreEqual(1, raw.A);
+            Assert.AreEqual(2, raw.B);
+
+            IPortableObject newPortObj = _grid.Portables().Builder(portObj).SetField("a", 3).Build();
+
+            raw = newPortObj.Deserialize<WithRaw>();
+
+            Assert.AreEqual(3, raw.A);
+            Assert.AreEqual(2, raw.B);
+        }
+
+        /// <summary>
+        /// Test nested objects.
+        /// </summary>
+        [Test]
+        public void TestNested()
+        {
+            // 1. Create from scratch.
+            IPortableBuilder builder = _grid.Portables().Builder(typeof(NestedOuter));
+
+            NestedInner inner1 = new NestedInner {Val = 1};
+            builder.SetField("inner1", inner1);
+
+            IPortableObject outerPortObj = builder.Build();
+
+            IPortableMetadata meta = outerPortObj.Metadata();
+
+            Assert.AreEqual(typeof(NestedOuter).Name, meta.TypeName);
+            Assert.AreEqual(1, meta.Fields.Count);
+            Assert.AreEqual(PortableTypeNames.TypeNameObject, meta.FieldTypeName("inner1"));
+
+            IPortableObject innerPortObj1 = outerPortObj.Field<IPortableObject>("inner1");
+
+            IPortableMetadata innerMeta = innerPortObj1.Metadata();
+
+            Assert.AreEqual(typeof(NestedInner).Name, innerMeta.TypeName);
+            Assert.AreEqual(1, innerMeta.Fields.Count);
+            Assert.AreEqual(PortableTypeNames.TypeNameInt, innerMeta.FieldTypeName("Val"));
+
+            inner1 = innerPortObj1.Deserialize<NestedInner>();
+
+            Assert.AreEqual(1, inner1.Val);
+
+            NestedOuter outer = outerPortObj.Deserialize<NestedOuter>();
+            Assert.AreEqual(outer.Inner1.Val, 1);
+            Assert.IsNull(outer.Inner2);
+
+            // 2. Add another field over existing portable object.
+            builder = _grid.Portables().Builder(outerPortObj);
+
+            NestedInner inner2 = new NestedInner {Val = 2};
+            builder.SetField("inner2", inner2);
+
+            outerPortObj = builder.Build();
+
+            outer = outerPortObj.Deserialize<NestedOuter>();
+            Assert.AreEqual(1, outer.Inner1.Val);
+            Assert.AreEqual(2, outer.Inner2.Val);
+
+            // 3. Try setting inner object in portable form.
+            innerPortObj1 = _grid.Portables().Builder(innerPortObj1).SetField("val", 3).Build();
+
+            inner1 = innerPortObj1.Deserialize<NestedInner>();
+
+            Assert.AreEqual(3, inner1.Val);
+
+            outerPortObj = _grid.Portables().Builder(outerPortObj).SetField<object>("inner1", innerPortObj1).Build();
+
+            outer = outerPortObj.Deserialize<NestedOuter>();
+            Assert.AreEqual(3, outer.Inner1.Val);
+            Assert.AreEqual(2, outer.Inner2.Val);
+        }
+
+        /// <summary>
+        /// Test handle migration.
+        /// </summary>
+        [Test]
+        public void TestHandleMigration()
+        {
+            // 1. Simple comparison of results.
+            MigrationInner inner = new MigrationInner {Val = 1};
+
+            MigrationOuter outer = new MigrationOuter
+            {
+                Inner1 = inner,
+                Inner2 = inner
+            };
+
+            byte[] outerBytes = _marsh.Marshal(outer);
+
+            IPortableBuilder builder = _grid.Portables().Builder(typeof(MigrationOuter));
+
+            builder.HashCode(outer.GetHashCode());
+
+            builder.SetField<object>("inner1", inner);
+            builder.SetField<object>("inner2", inner);
+
+            PortableUserObject portOuter = (PortableUserObject) builder.Build();
+
+            byte[] portOuterBytes = new byte[outerBytes.Length];
+
+            Buffer.BlockCopy(portOuter.Data, 0, portOuterBytes, 0, portOuterBytes.Length);
+
+            Assert.AreEqual(outerBytes, portOuterBytes);
+
+            // 2. Change the first inner object so that the handle must migrate.
+            MigrationInner inner1 = new MigrationInner {Val = 2};
+
+            IPortableObject portOuterMigrated =
+                _grid.Portables().Builder(portOuter).SetField<object>("inner1", inner1).Build();
+
+            MigrationOuter outerMigrated = portOuterMigrated.Deserialize<MigrationOuter>();
+
+            Assert.AreEqual(2, outerMigrated.Inner1.Val);
+            Assert.AreEqual(1, outerMigrated.Inner2.Val);
+
+            // 3. Change the first value using serialized form.
+            IPortableObject inner1Port =
+                _grid.Portables().Builder(typeof(MigrationInner)).SetField("val", 2).Build();
+
+            portOuterMigrated =
+                _grid.Portables().Builder(portOuter).SetField<object>("inner1", inner1Port).Build();
+
+            outerMigrated = portOuterMigrated.Deserialize<MigrationOuter>();
+
+            Assert.AreEqual(2, outerMigrated.Inner1.Val);
+            Assert.AreEqual(1, outerMigrated.Inner2.Val);
+        }
+
+        /// <summary>
+        /// Test handle inversion.
+        /// </summary>
+        [Test]
+        public void TestHandleInversion()
+        {
+            InversionInner inner = new InversionInner();
+            InversionOuter outer = new InversionOuter();
+
+            inner.Outer = outer;
+            outer.Inner = inner;
+
+            byte[] rawOuter = _marsh.Marshal(outer);
+
+            IPortableObject portOuter = _marsh.Unmarshal<IPortableObject>(rawOuter, PortableMode.ForcePortable);
+            IPortableObject portInner = portOuter.Field<IPortableObject>("inner");
+
+            // 1. Ensure that inner object can be deserialized after build.
+            IPortableObject portInnerNew = _grid.Portables().Builder(portInner).Build();
+
+            InversionInner innerNew = portInnerNew.Deserialize<InversionInner>();
+
+            Assert.AreSame(innerNew, innerNew.Outer.Inner);
+
+            // 2. Ensure that portable object with external dependencies could be added to builder.
+            IPortableObject portOuterNew =
+                _grid.Portables().Builder(typeof(InversionOuter)).SetField<object>("inner", portInner).Build();
+
+            InversionOuter outerNew = portOuterNew.Deserialize<InversionOuter>();
+
+            Assert.AreNotSame(outerNew, outerNew.Inner.Outer);
+            Assert.AreSame(outerNew.Inner, outerNew.Inner.Outer.Inner);
+        }
+
+        /// <summary>
+        /// Test build multiple objects.
+        /// </summary>
+        [Test]
+        public void TestBuildMultiple()
+        {
+            IPortableBuilder builder = _grid.Portables().Builder(typeof(Primitives));
+
+            builder.SetField<byte>("fByte", 1).SetField("fBool", true);
+
+            IPortableObject po1 = builder.Build();
+            IPortableObject po2 = builder.Build();
+
+            Assert.AreEqual(1, po1.Field<byte>("fByte"));
+            Assert.AreEqual(true, po1.Field<bool>("fBool"));
+
+            Assert.AreEqual(1, po2.Field<byte>("fByte"));
+            Assert.AreEqual(true, po2.Field<bool>("fBool"));
+
+            builder.SetField<byte>("fByte", 2);
+
+            IPortableObject po3 = builder.Build();
+
+            Assert.AreEqual(1, po1.Field<byte>("fByte"));
+            Assert.AreEqual(true, po1.Field<bool>("fBool"));
+
+            Assert.AreEqual(1, po2.Field<byte>("fByte"));
+            Assert.AreEqual(true, po2.Field<bool>("fBool"));
+
+            Assert.AreEqual(2, po3.Field<byte>("fByte"));
+            Assert.AreEqual(true, po2.Field<bool>("fBool"));
+
+            builder = _grid.Portables().Builder(po1);
+
+            builder.SetField<byte>("fByte", 10);
+
+            po1 = builder.Build();
+            po2 = builder.Build();
+
+            builder.SetField<byte>("fByte", 20);
+
+            po3 = builder.Build();
+
+            Assert.AreEqual(10, po1.Field<byte>("fByte"));
+            Assert.AreEqual(true, po1.Field<bool>("fBool"));
+
+            Assert.AreEqual(10, po2.Field<byte>("fByte"));
+            Assert.AreEqual(true, po2.Field<bool>("fBool"));
+
+            Assert.AreEqual(20, po3.Field<byte>("fByte"));
+            Assert.AreEqual(true, po3.Field<bool>("fBool"));
+        }
+
+        /// <summary>
+        /// Tests type id method.
+        /// </summary>
+        [Test]
+        public void TestTypeId()
+        {
+            Assert.Throws<ArgumentException>(() => _grid.Portables().GetTypeId(null));
+
+            Assert.AreEqual(IdMapper.TestTypeId, _grid.Portables().GetTypeId(IdMapper.TestTypeName));
+            
+            Assert.AreEqual(PortableUtils.StringHashCode("someTypeName"), _grid.Portables().GetTypeId("someTypeName"));
+        }
+
+        /// <summary>
+        /// Tests metadata methods.
+        /// </summary>
+        [Test]
+        public void TestMetadata()
+        {
+            // Populate metadata
+            var portables = _grid.Portables();
+
+            portables.ToPortable<IPortableObject>(new DecimalHolder());
+
+            // All meta
+            var allMetas = portables.GetMetadata();
+
+            var decimalMeta = allMetas.Single(x => x.TypeName == "DecimalHolder");
+
+            Assert.AreEqual(new[] {"val", "valArr"}, decimalMeta.Fields);
+
+            // By type
+            decimalMeta = portables.GetMetadata(typeof (DecimalHolder));
+
+            Assert.AreEqual(new[] {"val", "valArr"}, decimalMeta.Fields);
+            
+            // By type id
+            decimalMeta = portables.GetMetadata(portables.GetTypeId("DecimalHolder"));
+
+            Assert.AreEqual(new[] {"val", "valArr"}, decimalMeta.Fields);
+
+            // By type name
+            decimalMeta = portables.GetMetadata("DecimalHolder");
+
+            Assert.AreEqual(new[] {"val", "valArr"}, decimalMeta.Fields);
+        }
+
+        /// <summary>
+        /// Create portable type configuration with disabled metadata.
+        /// </summary>
+        /// <param name="typ">Type.</param>
+        /// <returns>Configuration.</returns>
+        private static PortableTypeConfiguration TypeConfigurationNoMeta(Type typ)
+        {
+            return new PortableTypeConfiguration(typ) {MetadataEnabled = false};
+        }
+    }
+
+    /// <summary>
+    /// Empty portable class.
+    /// </summary>
+    public class Empty
+    {
+        // No-op.
+    }
+
+    /// <summary>
+    /// Empty portable class with no metadata.
+    /// </summary>
+    public class EmptyNoMeta
+    {
+        // No-op.
+    }
+
+    /// <summary>
+    /// Portable with primitive fields.
+    /// </summary>
+    public class Primitives
+    {
+        public byte FByte;
+        public bool FBool;
+        public short FShort;
+        public char FChar;
+        public int FInt;
+        public long FLong;
+        public float FFloat;
+        public double FDouble;
+    }
+
+    /// <summary>
+    /// Portable with primitive array fields.
+    /// </summary>
+    public class PrimitiveArrays
+    {
+        public byte[] FByte;
+        public bool[] FBool;
+        public short[] FShort;
+        public char[] FChar;
+        public int[] FInt;
+        public long[] FLong;
+        public float[] FFloat;
+        public double[] FDouble;
+    }
+
+    /// <summary>
+    /// Portable having strings, dates, Guids and enums.
+    /// </summary>
+    public class StringDateGuidEnum
+    {
+        public string FStr;
+        public DateTime FDate;
+        public DateTime? FnDate;
+        public Guid FGuid;
+        public Guid? FnGuid;
+        public TestEnum FEnum;
+
+        public string[] FStrArr;
+        public DateTime?[] FDateArr;
+        public Guid?[] FGuidArr;
+        public TestEnum[] FEnumArr;
+    }
+
+    /// <summary>
+    /// Enumeration.
+    /// </summary>
+    public enum TestEnum
+    {
+        One, Two
+    }
+
+    /// <summary>
+    /// Portable with raw data.
+    /// </summary>
+    public class WithRaw : IPortableMarshalAware
+    {
+        public int A;
+        public int B;
+
+        /** <inheritDoc /> */
+        public void WritePortable(IPortableWriter writer)
+        {
+            writer.WriteInt("a", A);
+            writer.RawWriter().WriteInt(B);
+        }
+
+        /** <inheritDoc /> */
+        public void ReadPortable(IPortableReader reader)
+        {
+            A = reader.ReadInt("a");
+            B = reader.RawReader().ReadInt();
+        }
+    }
+
+    /// <summary>
+    /// Empty class for metadata overwrite test.
+    /// </summary>
+    public class MetaOverwrite
+    {
+        // No-op.
+    }
+
+    /// <summary>
+    /// Nested outer object.
+    /// </summary>
+    public class NestedOuter
+    {
+        public NestedInner Inner1;
+        public NestedInner Inner2;
+    }
+
+    /// <summary>
+    /// Nested inner object.
+    /// </summary>
+    public class NestedInner
+    {
+        public int Val;
+    }
+
+    /// <summary>
+    /// Outer object for handle migration test.
+    /// </summary>
+    public class MigrationOuter
+    {
+        public MigrationInner Inner1;
+        public MigrationInner Inner2;
+    }
+
+    /// <summary>
+    /// Inner object for handle migration test.
+    /// </summary>
+    public class MigrationInner
+    {
+        public int Val;
+    }
+
+    /// <summary>
+    /// Outer object for handle inversion test.
+    /// </summary>
+    public class InversionOuter
+    {
+        public InversionInner Inner;
+    }
+
+    /// <summary>
+    /// Inner object for handle inversion test.
+    /// </summary>
+    public class InversionInner
+    {
+        public InversionOuter Outer;
+    }
+
+    /// <summary>
+    /// Object for composite array tests.
+    /// </summary>
+    public class CompositeArray
+    {
+        public CompositeInner[] InArr;
+        public CompositeOuter[] OutArr;
+    }
+
+    /// <summary>
+    /// Object for composite collection/dictionary tests.
+    /// </summary>
+    public class CompositeContainer
+    {
+        public ICollection Col;
+        public ICollection<CompositeInner> GCol;
+
+        public IDictionary Dict;
+        public IDictionary<int, CompositeInner> GDict;
+    }
+
+    /// <summary>
+    /// OUter object for composite structures test.
+    /// </summary>
+    public class CompositeOuter
+    {
+        public CompositeInner Inner;
+
+        public CompositeOuter()
+        {
+            // No-op.
+        }
+
+        public CompositeOuter(CompositeInner inner)
+        {
+            Inner = inner;
+        }
+    }
+
+    /// <summary>
+    /// Inner object for composite structures test.
+    /// </summary>
+    public class CompositeInner
+    {
+        public int Val;
+
+        public CompositeInner()
+        {
+            // No-op.
+        }
+
+        public CompositeInner(int val)
+        {
+            Val = val;
+        }
+    }
+
+    /// <summary>
+    /// Type to test "ToPortable()" logic.
+    /// </summary>
+    public class ToPortable
+    {
+        public int Val;
+
+        public ToPortable(int val)
+        {
+            Val = val;
+        }
+    }
+
+    /// <summary>
+    /// Type to test "ToPortable()" logic with metadata disabled.
+    /// </summary>
+    public class ToPortableNoMeta
+    {
+        public int Val;
+
+        public ToPortableNoMeta(int val)
+        {
+            Val = val;
+        }
+    }
+
+    /// <summary>
+    /// Type to test removal.
+    /// </summary>
+    public class Remove
+    {
+        public object Val;
+        public RemoveInner Val2;
+    }
+
+    /// <summary>
+    /// Inner type to test removal.
+    /// </summary>
+    public class RemoveInner
+    {
+        /** */
+        public int Val;
+
+        /// <summary>
+        ///
+        /// </summary>
+        /// <param name="val"></param>
+        public RemoveInner(int val)
+        {
+            Val = val;
+        }
+    }
+
+    /// <summary>
+    ///
+    /// </summary>
+    public class BuilderInBuilderOuter
+    {
+        /** */
+        public BuilderInBuilderInner Inner;
+
+        /** */
+        public BuilderInBuilderInner Inner2;
+    }
+
+    /// <summary>
+    ///
+    /// </summary>
+    public class BuilderInBuilderInner
+    {
+        /** */
+        public BuilderInBuilderOuter Outer;
+    }
+
+    /// <summary>
+    ///
+    /// </summary>
+    public class BuilderCollection
+    {
+        /** */
+        public ICollection<BuilderCollectionItem> Col;
+
+        /// <summary>
+        ///
+        /// </summary>
+        /// <param name="col"></param>
+        public BuilderCollection(ICollection<BuilderCollectionItem> col)
+        {
+            Col = col;
+        }
+    }
+
+    /// <summary>
+    ///
+    /// </summary>
+    public class BuilderCollectionItem
+    {
+        /** */
+        public int Val;
+
+        /// <summary>
+        ///
+        /// </summary>
+        /// <param name="val"></param>
+        public BuilderCollectionItem(int val)
+        {
+            Val = val;
+        }
+    }
+
+    /// <summary>
+    ///
+    /// </summary>
+    public class DecimalHolder
+    {
+        /** */
+        public decimal Val;
+
+        /** */
+        public decimal[] ValArr;
+    }
+
+    /// <summary>
+    /// Test id mapper.
+    /// </summary>
+    public class IdMapper : IPortableIdMapper
+    {
+        /** */
+        public const string TestTypeName = "IdMapperTestType";
+
+        /** */
+        public const int TestTypeId = -65537;
+
+        /** <inheritdoc /> */
+        public int TypeId(string typeName)
+        {
+            return typeName == TestTypeName ? TestTypeId : 0;
+        }
+
+        /** <inheritdoc /> */
+        public int FieldId(int typeId, string fieldName)
+        {
+            return 0;
+        }
+    }
+}


[30/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/DelegateTypeDescriptor.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/DelegateTypeDescriptor.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/DelegateTypeDescriptor.cs
new file mode 100644
index 0000000..680228d
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/DelegateTypeDescriptor.cs
@@ -0,0 +1,327 @@
+/*
+ * 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 Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Datastream;
+    using Apache.Ignite.Core.Events;
+    using Apache.Ignite.Core.Impl.Cache;
+    using Apache.Ignite.Core.Impl.Datastream;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+    using Apache.Ignite.Core.Messaging;
+
+    /// <summary>
+    /// Type descriptor with precompiled delegates for known methods.
+    /// </summary>
+    internal class DelegateTypeDescriptor
+    {
+        /** Cached decriptors. */
+        private static readonly CopyOnWriteConcurrentDictionary<Type, DelegateTypeDescriptor> Descriptors 
+            = new CopyOnWriteConcurrentDictionary<Type, DelegateTypeDescriptor>();
+
+        /** */
+        private readonly Func<object, object> _computeOutFunc;
+
+        /** */
+        private readonly Func<object, object, object> _computeFunc;
+
+        /** */
+        private readonly Func<object, Guid, object, bool> _eventFilter;
+
+        /** */
+        private readonly Func<object, object, object, bool> _cacheEntryFilter;
+
+        /** */
+        private readonly Func<object, object, object, byte, bool> _cacheDrEntryFilter;
+
+        /** */
+        private readonly Tuple<Func<object, IMutableCacheEntryInternal, object, object>, Tuple<Type, Type>> 
+            _cacheEntryProcessor;
+
+        /** */
+        private readonly Func<object, Guid, object, bool> _messageFilter;
+
+        /** */
+        private readonly Func<object, object> _computeJobExecute;
+
+        /** */
+        private readonly Action<object> _computeJobCancel;
+
+        /** */
+        private readonly Action<object, Ignite, IUnmanagedTarget, IPortableStream, bool> _streamReceiver;
+
+        /** */
+        private readonly Func<object, object> _streamTransformerCtor;
+
+        /// <summary>
+        /// Gets the <see cref="IComputeFunc{T}" /> invocator.
+        /// </summary>
+        /// <param name="type">Type.</param>
+        /// <returns>Precompiled invocator delegate.</returns>
+        public static Func<object, object> GetComputeOutFunc(Type type)
+        {
+            return Get(type)._computeOutFunc;
+        }
+
+        /// <summary>
+        /// Gets the <see cref="IComputeFunc{T, R}" /> invocator.
+        /// </summary>
+        /// <param name="type">Type.</param>
+        /// <returns>Precompiled invocator delegate.</returns>
+        public static Func<object, object, object> GetComputeFunc(Type type)
+        {
+            return Get(type)._computeFunc;
+        }
+
+        /// <summary>
+        /// Gets the <see cref="IEventFilter{T}" /> invocator.
+        /// </summary>
+        /// <param name="type">Type.</param>
+        /// <returns>Precompiled invocator delegate.</returns>
+        public static Func<object, Guid, object, bool> GetEventFilter(Type type)
+        {
+            return Get(type)._eventFilter;
+        }
+
+        /// <summary>
+        /// Gets the <see cref="ICacheEntryFilter{TK,TV}" /> invocator.
+        /// </summary>
+        /// <param name="type">Type.</param>
+        /// <returns>Precompiled invocator delegate.</returns>
+        public static Func<object, object, object, bool> GetCacheEntryFilter(Type type)
+        {
+            return Get(type)._cacheEntryFilter;
+        }
+
+        /// <summary>
+        /// Gets the <see cref="ICacheDrEntryFilter{K, V}" /> invocator.
+        /// </summary>
+        /// <param name="type">Type.</param>
+        /// <returns>Precompiled invocator delegate.</returns>
+        public static Func<object, object, object, byte, bool> GetCacheDrEntryFilter(Type type)
+        {
+            return Get(type)._cacheDrEntryFilter;
+        }
+
+        /// <summary>
+        /// Gets the <see cref="ICacheEntryProcessor{K, V, A, R}" /> invocator.
+        /// </summary>
+        /// <param name="type">Type.</param>
+        /// <returns>Precompiled invocator delegate.</returns>
+        public static Func<object, IMutableCacheEntryInternal, object, object> GetCacheEntryProcessor(Type type)
+        {
+            return Get(type)._cacheEntryProcessor.Item1;
+        }
+
+        /// <summary>
+        /// Gets key and value types for the <see cref="ICacheEntryProcessor{K, V, A, R}" />.
+        /// </summary>
+        /// <param name="type">Type.</param>
+        /// <returns>Key and value types.</returns>
+        public static Tuple<Type, Type> GetCacheEntryProcessorTypes(Type type)
+        {
+            return Get(type)._cacheEntryProcessor.Item2;
+        }
+
+        /// <summary>
+        /// Gets the <see cref="IMessageFilter{T}" /> invocator.
+        /// </summary>
+        /// <param name="type">Type.</param>
+        /// <returns>Precompiled invocator delegate.</returns>
+        public static Func<object, Guid, object, bool> GetMessageFilter(Type type)
+        {
+            return Get(type)._messageFilter;
+        }
+
+        /// <summary>
+        /// Gets the <see cref="IComputeJob{T}.Execute" /> and <see cref="IComputeJob{T}.Cancel" /> invocators.
+        /// </summary>
+        /// <param name="type">Type.</param>
+        /// <param name="execute">Execute invocator.</param>
+        /// <param name="cancel">Cancel invocator.</param>
+        public static void GetComputeJob(Type type, out Func<object, object> execute, out Action<object> cancel)
+        {
+            var desc = Get(type);
+
+            execute = desc._computeJobExecute;
+            cancel = desc._computeJobCancel;
+        }
+
+        /// <summary>
+        /// Gets the <see cref="IStreamReceiver{TK,TV}"/> invocator.
+        /// </summary>
+        /// <param name="type">Type.</param>
+        /// <returns>Precompiled invocator delegate.</returns>
+        public static Action<object, Ignite, IUnmanagedTarget, IPortableStream, bool> GetStreamReceiver(Type type)
+        {
+            return Get(type)._streamReceiver;
+        }
+
+        /// <summary>
+        /// Gets the <see cref="StreamTransformer{K, V, A, R}"/>> ctor invocator.
+        /// </summary>
+        /// <param name="type">Type.</param>
+        /// <returns>Precompiled invocator delegate.</returns>
+        public static Func<object, object> GetStreamTransformerCtor(Type type)
+        {
+            return Get(type)._streamTransformerCtor;
+        }
+
+        /// <summary>
+        /// Gets the <see cref="DelegateTypeDescriptor" /> by type.
+        /// </summary>
+        private static DelegateTypeDescriptor Get(Type type)
+        {
+            DelegateTypeDescriptor result;
+
+            return Descriptors.TryGetValue(type, out result)
+                ? result
+                : Descriptors.GetOrAdd(type, t => new DelegateTypeDescriptor(t));
+        }
+
+        /// <summary>
+        /// Throws an exception if first argument is not null.
+        /// </summary>
+        // ReSharper disable once UnusedParameter.Local
+        private static void ThrowIfMultipleInterfaces(object check, Type userType, Type interfaceType)
+        {
+            if (check != null)
+                throw new InvalidOperationException(
+                    string.Format("Not Supported: Type {0} implements interface {1} multiple times.", userType,
+                        interfaceType));
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="DelegateTypeDescriptor"/> class.
+        /// </summary>
+        /// <param name="type">The type.</param>
+        private DelegateTypeDescriptor(Type type)
+        {
+            foreach (var iface in type.GetInterfaces())
+            {
+                if (!iface.IsGenericType)
+                    continue;
+
+                var genericTypeDefinition = iface.GetGenericTypeDefinition();
+
+                if (genericTypeDefinition == typeof (IComputeFunc<>))
+                {
+                    ThrowIfMultipleInterfaces(_computeOutFunc, type, typeof(IComputeFunc<>));
+
+                    _computeOutFunc = DelegateConverter.CompileFunc(iface);
+                }
+                else if (genericTypeDefinition == typeof (IComputeFunc<,>))
+                {
+                    ThrowIfMultipleInterfaces(_computeFunc, type, typeof(IComputeFunc<,>));
+
+                    var args = iface.GetGenericArguments();
+
+                    _computeFunc = DelegateConverter.CompileFunc<Func<object, object, object>>(iface, new[] {args[0]});
+                }
+                else if (genericTypeDefinition == typeof (IEventFilter<>))
+                {
+                    ThrowIfMultipleInterfaces(_eventFilter, type, typeof(IEventFilter<>));
+
+                    var args = iface.GetGenericArguments();
+
+                    _eventFilter = DelegateConverter.CompileFunc<Func<object, Guid, object, bool>>(iface, 
+                        new[] {typeof (Guid), args[0]}, new[] {false, true, false});
+                }
+                else if (genericTypeDefinition == typeof (ICacheEntryFilter<,>))
+                {
+                    ThrowIfMultipleInterfaces(_cacheEntryFilter, type, typeof(ICacheEntryFilter<,>));
+
+                    var args = iface.GetGenericArguments();
+
+                    var entryType = typeof (ICacheEntry<,>).MakeGenericType(args);
+
+                    var invokeFunc = DelegateConverter.CompileFunc<Func<object, object, bool>>(iface,
+                        new[] { entryType }, new[] { true, false });
+
+                    var ctor = DelegateConverter.CompileCtor<Func<object, object, object>>(
+                            typeof (CacheEntry<,>).MakeGenericType(args), args);
+
+                    // Resulting func constructs CacheEntry and passes it to user implementation
+                    _cacheEntryFilter = (obj, k, v) => invokeFunc(obj, ctor(k, v));
+                }
+                else if (genericTypeDefinition == typeof (ICacheEntryProcessor<,,,>))
+                {
+                    ThrowIfMultipleInterfaces(_cacheEntryProcessor, type, typeof(ICacheEntryProcessor<,,,>));
+
+                    var args = iface.GetGenericArguments();
+
+                    var entryType = typeof (IMutableCacheEntry<,>).MakeGenericType(args[0], args[1]);
+
+                    var func = DelegateConverter.CompileFunc<Func<object, object, object, object>>(iface,
+                        new[] { entryType, args[2] }, null, "Process");
+
+                    var types = new Tuple<Type, Type>(args[0], args[1]);
+
+                    _cacheEntryProcessor = new Tuple<Func<object, IMutableCacheEntryInternal, object, object>, Tuple<Type, Type>>
+                        (func, types);
+
+                    var transformerType = typeof (StreamTransformer<,,,>).MakeGenericType(args);
+
+                    _streamTransformerCtor = DelegateConverter.CompileCtor<Func<object, object>>(transformerType,
+                        new[] {iface});
+                }
+                else if (genericTypeDefinition == typeof (IMessageFilter<>))
+                {
+                    ThrowIfMultipleInterfaces(_messageFilter, type, typeof(IMessageFilter<>));
+
+                    var arg = iface.GetGenericArguments()[0];
+
+                    _messageFilter = DelegateConverter.CompileFunc<Func<object, Guid, object, bool>>(iface,
+                        new[] { typeof(Guid), arg }, new[] { false, true, false });
+                }
+                else if (genericTypeDefinition == typeof (IComputeJob<>))
+                {
+                    ThrowIfMultipleInterfaces(_messageFilter, type, typeof(IComputeJob<>));
+
+                    _computeJobExecute = DelegateConverter.CompileFunc<Func<object, object>>(iface, new Type[0], 
+                        methodName: "Execute");
+
+                    _computeJobCancel = DelegateConverter.CompileFunc<Action<object>>(iface, new Type[0],
+                        new[] {false}, "Cancel");
+                }
+                else if (genericTypeDefinition == typeof (IStreamReceiver<,>))
+                {
+                    ThrowIfMultipleInterfaces(_streamReceiver, type, typeof (IStreamReceiver<,>));
+
+                    var method =
+                        typeof (StreamReceiverHolder).GetMethod("InvokeReceiver")
+                            .MakeGenericMethod(iface.GetGenericArguments());
+
+                    _streamReceiver = DelegateConverter
+                        .CompileFunc<Action<object, Ignite, IUnmanagedTarget, IPortableStream, bool>>(
+                            typeof (StreamReceiverHolder),
+                            method,
+                            new[]
+                            {
+                                iface, typeof (Ignite), typeof (IUnmanagedTarget), typeof (IPortableStream),
+                                typeof (bool)
+                            },
+                            new[] {true, false, false, false, false, false});
+                }
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
index c62cfd2..0bbc1a2 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
@@ -22,7 +22,6 @@ namespace Apache.Ignite.Core.Impl.Common
     using System.Diagnostics.CodeAnalysis;
     using System.Threading;
     using System.Threading.Tasks;
-    
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Impl.Portable.IO;
 
@@ -133,7 +132,7 @@ namespace Apache.Ignite.Core.Impl.Common
         /** <inheritdoc/> */
         public void Listen(Action<IFuture<T>> callback)
         {
-            GridArgumentCheck.NotNull(callback, "callback");
+            IgniteArgumentCheck.NotNull(callback, "callback");
 
             if (!_done)
             {

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/FutureConverter.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/FutureConverter.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/FutureConverter.cs
new file mode 100644
index 0000000..a07d954
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/FutureConverter.cs
@@ -0,0 +1,62 @@
+/*
+ * 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 Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+
+    /// <summary>
+    /// Marshals and converts future value.
+    /// </summary>
+    internal class FutureConverter<T> : IFutureConverter<T>
+    {
+        /** Marshaller. */
+        private readonly PortableMarshaller _marsh;
+
+        /** Keep portable flag. */
+        private readonly bool _keepPortable;
+
+        /** Converting function. */
+        private readonly Func<PortableReaderImpl, T> _func;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="marsh">Marshaller.</param>
+        /// <param name="keepPortable">Keep portable.</param>
+        /// <param name="func">Converting function.</param>
+        public FutureConverter(PortableMarshaller marsh, bool keepPortable,
+            Func<PortableReaderImpl, T> func = null)
+        {
+            _marsh = marsh;
+            _keepPortable = keepPortable;
+            _func = func ?? (reader => reader.ReadObject<T>());
+        }
+
+        /// <summary>
+        /// Read and convert a value.
+        /// </summary>
+        public T Convert(IPortableStream stream)
+        {
+            var reader = _marsh.StartUnmarshal(stream, _keepPortable);
+
+            return _func(reader);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/GridArgumentCheck.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/GridArgumentCheck.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/GridArgumentCheck.cs
deleted file mode 100644
index a1fadfe..0000000
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/GridArgumentCheck.cs
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * 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.Generic;
-
-    /// <summary>
-    /// Arguments check helpers.
-    /// </summary>
-    public static class GridArgumentCheck
-    {
-        /// <summary>
-        /// Throws an ArgumentNullException if specified arg is null.
-        /// </summary>
-        /// <param name="arg">The argument.</param>
-        /// <param name="argName">Name of the argument.</param>
-        public static void NotNull(object arg, string argName)
-        {
-            if (arg == null)
-                throw new ArgumentNullException(argName);
-        }
-
-        /// <summary>
-        /// Throws an ArgumentException if specified arg is null or empty string.
-        /// </summary>
-        /// <param name="arg">The argument.</param>
-        /// <param name="argName">Name of the argument.</param>
-        public static void NotNullOrEmpty(string arg, string argName)
-        {
-            if (string.IsNullOrEmpty(arg))
-                throw new ArgumentException(string.Format("'{0}' argument should not be null or empty.", argName),
-                    argName);
-        }
-
-        /// <summary>
-        /// Throws an ArgumentException if specified arg is null or empty string.
-        /// </summary>
-        /// <param name="collection">The collection.</param>
-        /// <param name="argName">Name of the argument.</param>
-        public static void NotNullOrEmpty<T>(ICollection<T> collection, string argName)
-        {
-            if (collection == null || collection.Count == 0)
-                throw new ArgumentException(string.Format("'{0}' argument should not be null or empty.", argName),
-                    argName);
-        }
-
-        /// <summary>
-        /// Throws an ArgumentException if specified condition is false.
-        /// </summary>
-        /// <param name="condition">Condition.</param>
-        /// <param name="argName">Name of the argument.</param>
-        /// <param name="message">Message.</param>
-        public static void Ensure(bool condition, string argName, string message)
-        {
-            if (!condition)
-                throw new ArgumentException(string.Format("'{0}' argument is invalid: {1}", argName, message), 
-                    argName);
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/IgniteArgumentCheck.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/IgniteArgumentCheck.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/IgniteArgumentCheck.cs
new file mode 100644
index 0000000..e94c577
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/IgniteArgumentCheck.cs
@@ -0,0 +1,76 @@
+/*
+ * 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.Generic;
+
+    /// <summary>
+    /// Arguments check helpers.
+    /// </summary>
+    public static class IgniteArgumentCheck
+    {
+        /// <summary>
+        /// Throws an ArgumentNullException if specified arg is null.
+        /// </summary>
+        /// <param name="arg">The argument.</param>
+        /// <param name="argName">Name of the argument.</param>
+        public static void NotNull(object arg, string argName)
+        {
+            if (arg == null)
+                throw new ArgumentNullException(argName);
+        }
+
+        /// <summary>
+        /// Throws an ArgumentException if specified arg is null or empty string.
+        /// </summary>
+        /// <param name="arg">The argument.</param>
+        /// <param name="argName">Name of the argument.</param>
+        public static void NotNullOrEmpty(string arg, string argName)
+        {
+            if (string.IsNullOrEmpty(arg))
+                throw new ArgumentException(string.Format("'{0}' argument should not be null or empty.", argName),
+                    argName);
+        }
+
+        /// <summary>
+        /// Throws an ArgumentException if specified arg is null or empty string.
+        /// </summary>
+        /// <param name="collection">The collection.</param>
+        /// <param name="argName">Name of the argument.</param>
+        public static void NotNullOrEmpty<T>(ICollection<T> collection, string argName)
+        {
+            if (collection == null || collection.Count == 0)
+                throw new ArgumentException(string.Format("'{0}' argument should not be null or empty.", argName),
+                    argName);
+        }
+
+        /// <summary>
+        /// Throws an ArgumentException if specified condition is false.
+        /// </summary>
+        /// <param name="condition">Condition.</param>
+        /// <param name="argName">Name of the argument.</param>
+        /// <param name="message">Message.</param>
+        public static void Ensure(bool condition, string argName, string message)
+        {
+            if (!condition)
+                throw new ArgumentException(string.Format("'{0}' argument is invalid: {1}", argName, message), 
+                    argName);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/PortableResultWrapper.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/PortableResultWrapper.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/PortableResultWrapper.cs
new file mode 100644
index 0000000..733bed0
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/PortableResultWrapper.cs
@@ -0,0 +1,68 @@
+/*
+ * 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 Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Simple wrapper over result to handle marshalling properly.
+    /// </summary>
+    internal class PortableResultWrapper : IPortableWriteAware
+    {
+        /** */
+        private readonly object _result;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="PortableResultWrapper"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        public PortableResultWrapper(IPortableReader reader)
+        {
+            var reader0 = (PortableReaderImpl)reader.RawReader();
+
+            _result = PortableUtils.ReadPortableOrSerializable<object>(reader0);
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="res">Result.</param>
+        public PortableResultWrapper(object res)
+        {
+            _result = res;
+        }
+
+        /// <summary>
+        /// Result.
+        /// </summary>
+        public object Result
+        {
+            get { return _result; }
+        }
+
+        /** <inheritDoc /> */
+        public void WritePortable(IPortableWriter writer)
+        {
+            var writer0 = (PortableWriterImpl) writer.RawWriter();
+
+            writer0.DetachNext();
+            PortableUtils.WritePortableOrSerializable(writer0, Result);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeAbstractClosureTask.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeAbstractClosureTask.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeAbstractClosureTask.cs
new file mode 100644
index 0000000..1a772c2
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeAbstractClosureTask.cs
@@ -0,0 +1,101 @@
+/*
+ * 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.Compute.Closure
+{
+    using System;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Compute;
+
+    /// <summary>
+    /// Base class for all tasks working with closures.
+    /// </summary>
+    internal abstract class ComputeAbstractClosureTask<TA, T, TR> : IComputeTask<TA, T, TR>
+    {
+        /// <summary>
+        /// This method is called to map or split Ignite task into multiple Ignite jobs. This is the
+        /// first method that gets called when task execution starts.
+        /// </summary>
+        /// <param name="subgrid">Nodes available for this task execution. Note that order of nodes is
+        /// guaranteed to be randomized by container. This ensures that every time you simply iterate
+        /// through Ignite nodes, the order of nodes will be random which over time should result into
+        /// all nodes being used equally.</param>
+        /// <param name="arg">Task execution argument. Can be <c>null</c>. This is the same argument
+        /// as the one passed into <c>ICompute.Execute()</c> methods.</param>
+        /// <returns>
+        /// Map of Ignite jobs assigned to subgrid node. If <c>null</c> or empty map is returned,
+        /// exception will be thrown.
+        /// </returns>
+        /// <exception cref="System.NotSupportedException">Map step should not be called on this task.</exception>
+        public IDictionary<IComputeJob<T>, IClusterNode> Map(IList<IClusterNode> subgrid, TA arg)
+        {
+            throw new NotSupportedException("Map step should not be called on this task.");
+        }
+
+        /// <summary>
+        /// Asynchronous callback invoked every time a result from remote execution is
+        /// received. It is ultimately upto this method to return a policy based
+        /// on which the system will either wait for more results, reduce results
+        /// received so far, or failover this job to another node. See
+        /// <see cref="ComputeJobResultPolicy" /> for more information.
+        /// </summary>
+        /// <param name="res">Received remote Ignite executable result.</param>
+        /// <param name="rcvd">All previously received results. Note that if task class has
+        /// <see cref="ComputeTaskNoResultCacheAttribute" /> attribute, then this list will be empty.</param>
+        /// <returns>
+        /// Result policy that dictates how to process further upcoming job results.
+        /// </returns>
+        public ComputeJobResultPolicy Result(IComputeJobResult<T> res, IList<IComputeJobResult<T>> rcvd)
+        {
+            Exception err = res.Exception();
+
+            if (err != null)
+            {
+                if (err is ComputeExecutionRejectedException || err is ClusterTopologyException || 
+                    err is ComputeJobFailoverException)
+                    return ComputeJobResultPolicy.Failover;
+                
+                throw err;
+            }
+            
+            return Result0(res);
+        }
+
+        /// <summary>
+        /// Reduces (or aggregates) results received so far into one compound result to be returned to
+        /// caller via future.
+        /// <para />
+        /// Note, that if some jobs did not succeed and could not be failed over then the list of
+        /// results passed into this method will include the failed results. Otherwise, failed
+        /// results will not be in the list.
+        /// </summary>
+        /// <param name="results">Received job results. Note that if task class has
+        /// <see cref="ComputeTaskNoResultCacheAttribute" /> attribute, then this list will be empty.</param>
+        /// <returns>
+        /// Task result constructed from results of remote executions.
+        /// </returns>
+        public abstract TR Reduce(IList<IComputeJobResult<T>> results);
+
+        /// <summary>
+        /// Internal result processing routine.
+        /// </summary>
+        /// <param name="res">Result.</param>
+        /// <returns>Policy.</returns>
+        protected abstract ComputeJobResultPolicy Result0(IComputeJobResult<T> res);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeActionJob.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeActionJob.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeActionJob.cs
new file mode 100644
index 0000000..c91a167
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeActionJob.cs
@@ -0,0 +1,83 @@
+/*
+ * 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.Compute.Closure
+{
+    using System;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Resource;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// System job which wraps over <c>Action</c>.
+    /// </summary>
+    internal class ComputeActionJob : IComputeJob, IComputeResourceInjector, IPortableWriteAware
+    {
+        /** Closure. */
+        private readonly IComputeAction _action;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="action">Action.</param>
+        public ComputeActionJob(IComputeAction action)
+        {
+            _action = action;
+        }
+
+        /** <inheritDoc /> */
+        public object Execute()
+        {
+            _action.Invoke();
+            
+            return null;
+        }
+
+        /** <inheritDoc /> */
+        public void Cancel()
+        {
+            throw new NotSupportedException("Func job cannot be cancelled.");
+        }
+
+        /** <inheritDoc /> */
+        public void Inject(Ignite grid)
+        {
+            ResourceProcessor.Inject(_action, grid);
+        }
+
+        /** <inheritDoc /> */
+        public void WritePortable(IPortableWriter writer)
+        {
+            var writer0 = (PortableWriterImpl)writer.RawWriter();
+
+            writer0.DetachNext();
+            PortableUtils.WritePortableOrSerializable(writer0, _action);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeActionJob"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        public ComputeActionJob(IPortableReader reader)
+        {
+            var reader0 = (PortableReaderImpl)reader.RawReader();
+
+            _action = PortableUtils.ReadPortableOrSerializable<IComputeAction>(reader0);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeFuncJob.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeFuncJob.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeFuncJob.cs
new file mode 100644
index 0000000..381c701
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeFuncJob.cs
@@ -0,0 +1,89 @@
+/*
+ * 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.Compute.Closure
+{
+    using System;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Resource;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// System job which wraps over <c>Func</c>.
+    /// </summary>
+    internal class ComputeFuncJob : IComputeJob, IComputeResourceInjector, IPortableWriteAware
+    {
+        /** Closure. */
+        private readonly IComputeFunc _clo;
+
+        /** Argument. */
+        private readonly object _arg;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="clo">Closure.</param>
+        /// <param name="arg">Argument.</param>
+        public ComputeFuncJob(IComputeFunc clo, object arg)
+        {
+            _clo = clo;
+            _arg = arg;
+        }
+
+        /** <inheritDoc /> */
+        public object Execute()
+        {
+            return _clo.Invoke(_arg);
+        }
+
+        /** <inheritDoc /> */
+        public void Cancel()
+        {
+            throw new NotSupportedException("Func job cannot be cancelled.");
+        }
+
+        /** <inheritDoc /> */
+        public void Inject(Ignite grid)
+        {
+            ResourceProcessor.Inject(_clo, grid);
+        }
+
+        /** <inheritDoc /> */
+        public void WritePortable(IPortableWriter writer)
+        {
+            PortableWriterImpl writer0 = (PortableWriterImpl) writer.RawWriter();
+
+            writer0.DetachNext();
+            PortableUtils.WritePortableOrSerializable(writer0, _clo);
+
+            writer0.DetachNext();
+            PortableUtils.WritePortableOrSerializable(writer0, _arg);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeFuncJob"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        public ComputeFuncJob(IPortableReader reader)
+        {
+            var reader0 = (PortableReaderImpl) reader.RawReader();
+            
+            _clo = PortableUtils.ReadPortableOrSerializable<IComputeFunc>(reader0);
+            _arg = PortableUtils.ReadPortableOrSerializable<object>(reader0);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeMultiClosureTask.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeMultiClosureTask.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeMultiClosureTask.cs
new file mode 100644
index 0000000..dd57f6c
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeMultiClosureTask.cs
@@ -0,0 +1,56 @@
+/*
+ * 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.Compute.Closure
+{
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Compute;
+
+    /// <summary>
+    /// Closure-based task producing multiple jobs and returning a collection of job results.
+    /// </summary>
+    [ComputeTaskNoResultCache]
+    internal class ComputeMultiClosureTask<TA, T, TR> : ComputeAbstractClosureTask<TA, T, TR> 
+        where TR : ICollection<T>
+    {
+        /** Result. */
+        private readonly ICollection<T> _res;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="size">Expected results count.</param>
+        public ComputeMultiClosureTask(int size)
+        {
+            _res = new List<T>(size);
+        }
+
+        /** <inheritDoc /> */
+        protected override ComputeJobResultPolicy Result0(IComputeJobResult<T> res)
+        {
+            _res.Add(res.Data());
+
+            return ComputeJobResultPolicy.Wait;
+        }
+
+        /** <inheritDoc /> */
+        public override TR Reduce(IList<IComputeJobResult<T>> results)
+        {
+            return (TR) _res;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeOutFuncJob.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeOutFuncJob.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeOutFuncJob.cs
new file mode 100644
index 0000000..5f719cd
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeOutFuncJob.cs
@@ -0,0 +1,76 @@
+/*
+ * 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.Compute.Closure
+{
+    using System;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Resource;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// System job which wraps over <c>Func</c>.
+    /// </summary>
+    internal class ComputeOutFuncJob : IComputeJob, IComputeResourceInjector, IPortableWriteAware
+    {
+        /** Closure. */
+        private readonly IComputeOutFunc _clo;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="clo">Closure.</param>
+        public ComputeOutFuncJob(IComputeOutFunc clo)
+        {
+            _clo = clo;
+        }
+
+        /** <inheritDoc /> */
+        public object Execute()
+        {
+            return _clo.Invoke();
+        }
+
+        /** <inheritDoc /> */
+        public void Cancel()
+        {
+            throw new NotSupportedException("Func job cannot be cancelled.");
+        }
+
+        /** <inheritDoc /> */
+        public void Inject(Ignite grid)
+        {
+            ResourceProcessor.Inject(_clo, grid);
+        }
+
+        /** <inheritDoc /> */
+        public void WritePortable(IPortableWriter writer)
+        {
+            var writer0 = (PortableWriterImpl) writer.RawWriter();
+
+            writer0.DetachNext();
+            PortableUtils.WritePortableOrSerializable(writer0, _clo);
+        }
+
+        public ComputeOutFuncJob(IPortableReader reader)
+        {
+            var reader0 = (PortableReaderImpl) reader.RawReader();
+
+            _clo = PortableUtils.ReadPortableOrSerializable<IComputeOutFunc>(reader0);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeReducingClosureTask.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeReducingClosureTask.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeReducingClosureTask.cs
new file mode 100644
index 0000000..a84d7ce
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeReducingClosureTask.cs
@@ -0,0 +1,61 @@
+/*
+ * 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.Compute.Closure
+{
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Impl.Resource;
+
+    /// <summary>
+    /// Closure-based task producing only one job and thus having only single result.
+    /// </summary>
+    [ComputeTaskNoResultCache]
+    internal class ComputeReducingClosureTask<TA, T, TR> 
+        : ComputeAbstractClosureTask<TA, T, TR>, IComputeResourceInjector
+    {
+        /** Reducer. */
+        private readonly IComputeReducer<T, TR> _rdc;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="rdc">Reducer.</param>
+        public ComputeReducingClosureTask(IComputeReducer<T, TR> rdc)
+        {
+            _rdc = rdc;
+        }
+
+        /** <inheritDoc /> */
+        protected override ComputeJobResultPolicy Result0(IComputeJobResult<T> res)
+        {
+            return _rdc.Collect(res.Data()) ? ComputeJobResultPolicy.Wait : ComputeJobResultPolicy.Reduce;
+        }
+
+        /** <inheritDoc /> */
+        public override TR Reduce(IList<IComputeJobResult<T>> results)
+        {
+            return _rdc.Reduce();
+        }
+
+        /** <inheritDoc /> */
+        public void Inject(Ignite grid)
+        {
+            ResourceProcessor.Inject(_rdc, grid);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeSingleClosureTask.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeSingleClosureTask.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeSingleClosureTask.cs
new file mode 100644
index 0000000..6e82c9b
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeSingleClosureTask.cs
@@ -0,0 +1,48 @@
+/*
+ * 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.Compute.Closure
+{
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Compute;
+
+    /// <summary>
+    /// Closure-based task producing only one job and thus having only single result.
+    /// </summary>
+    [ComputeTaskNoResultCache]
+    internal class ComputeSingleClosureTask<TA, T, TR> : ComputeAbstractClosureTask<TA, T, TR> where TR : T
+    {
+        /** Result. */
+        private TR _res;
+
+        /** <inheritDoc /> */
+        protected override ComputeJobResultPolicy Result0(IComputeJobResult<T> res)
+        {
+            _res = (TR) res.Data();
+
+            // No more results are expected at this point, but we prefer not to alter regular
+            // task flow.
+            return ComputeJobResultPolicy.Wait;
+        }
+
+        /** <inheritDoc /> */
+        public override TR Reduce(IList<IComputeJobResult<T>> results)
+        {
+            return _res;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/IComputeResourceInjector.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/IComputeResourceInjector.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/IComputeResourceInjector.cs
new file mode 100644
index 0000000..8d3e8d7
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/IComputeResourceInjector.cs
@@ -0,0 +1,31 @@
+/*
+ * 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.Compute.Closure
+{
+    /// <summary>
+    /// Interface denoting entity which must perform custom resource injection.
+    /// </summary>
+    internal interface IComputeResourceInjector
+    {
+        /// <summary>
+        /// Inject resources.
+        /// </summary>
+        /// <param name="grid">Grid.</param>
+        void Inject(Ignite grid);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Compute.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Compute.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Compute.cs
new file mode 100644
index 0000000..7efabd1
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/Compute.cs
@@ -0,0 +1,213 @@
+/*
+ * 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.Compute
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Compute;
+
+    /// <summary>
+    /// Synchronous Compute facade.
+    /// </summary>
+    internal class Compute : ICompute
+    {
+        /** */
+        private readonly ComputeImpl _compute;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="Compute"/> class.
+        /// </summary>
+        /// <param name="computeImpl">The compute implementation.</param>
+        public Compute(ComputeImpl computeImpl)
+        {
+            Debug.Assert(computeImpl != null);
+
+            _compute = computeImpl;
+        }
+
+        /** <inheritDoc /> */
+        public ICompute WithAsync()
+        {
+            return new ComputeAsync(_compute);
+        }
+
+        /** <inheritDoc /> */
+        public bool IsAsync
+        {
+            get { return false; }
+        }
+
+        /** <inheritDoc /> */
+        public IFuture GetFuture()
+        {
+            throw IgniteUtils.GetAsyncModeDisabledException();
+        }
+
+        /** <inheritDoc /> */
+        public IFuture<TResult> GetFuture<TResult>()
+        {
+            throw IgniteUtils.GetAsyncModeDisabledException();
+        }
+
+        /** <inheritDoc /> */
+        public IClusterGroup ClusterGroup
+        {
+            get { return _compute.ClusterGroup; }
+        }
+
+        /** <inheritDoc /> */
+        public ICompute WithNoFailover()
+        {
+            _compute.WithNoFailover();
+
+            return this;
+        }
+
+        /** <inheritDoc /> */
+        public ICompute WithTimeout(long timeout)
+        {
+            _compute.WithTimeout(timeout);
+
+            return this;
+        }
+
+        /** <inheritDoc /> */
+        public ICompute WithKeepPortable()
+        {
+            _compute.WithKeepPortable();
+
+            return this;
+        }
+
+        /** <inheritDoc /> */
+        public T ExecuteJavaTask<T>(string taskName, object taskArg)
+        {
+            return _compute.ExecuteJavaTask<T>(taskName, taskArg);
+        }
+
+        /** <inheritDoc /> */
+        public TR Execute<TA, T, TR>(IComputeTask<TA, T, TR> task, TA taskArg)
+        {
+            return _compute.Execute(task, taskArg).Get();
+        }
+
+        /** <inheritDoc /> */
+        public TR Execute<T, TR>(IComputeTask<T, TR> task)
+        {
+            return _compute.Execute(task, null).Get();
+        }
+
+        /** <inheritDoc /> */
+        public TR Execute<TA, T, TR>(Type taskType, TA taskArg)
+        {
+            return _compute.Execute<TA, T, TR>(taskType, taskArg).Get();
+        }
+
+        public TR Execute<T, TR>(Type taskType)
+        {
+            return _compute.Execute<object, T, TR>(taskType, null).Get();
+        }
+
+        /** <inheritDoc /> */
+        public TR Call<TR>(IComputeFunc<TR> clo)
+        {
+            return _compute.Execute(clo).Get();
+        }
+
+        /** <inheritDoc /> */
+        public TR AffinityCall<TR>(string cacheName, object affinityKey, IComputeFunc<TR> clo)
+        {
+            return _compute.AffinityCall(cacheName, affinityKey, clo).Get();
+        }
+
+        /** <inheritDoc /> */
+        public TR Call<TR>(Func<TR> func)
+        {
+            return _compute.Execute(func).Get();
+        }
+
+        /** <inheritDoc /> */
+        public ICollection<TR> Call<TR>(IEnumerable<IComputeFunc<TR>> clos)
+        {
+            return _compute.Execute(clos).Get();
+        }
+
+        /** <inheritDoc /> */
+        public TR2 Call<TR1, TR2>(IEnumerable<IComputeFunc<TR1>> clos, IComputeReducer<TR1, TR2> rdc)
+        {
+            return _compute.Execute(clos, rdc).Get();
+        }
+
+        /** <inheritDoc /> */
+        public ICollection<TR> Broadcast<TR>(IComputeFunc<TR> clo)
+        {
+            return _compute.Broadcast(clo).Get();
+        }
+
+        /** <inheritDoc /> */
+        public ICollection<TR> Broadcast<T, TR>(IComputeFunc<T, TR> clo, T arg)
+        {
+            return _compute.Broadcast(clo, arg).Get();
+        }
+
+        /** <inheritDoc /> */
+        public void Broadcast(IComputeAction action)
+        {
+            _compute.Broadcast(action).Get();
+        }
+
+        /** <inheritDoc /> */
+        public void Run(IComputeAction action)
+        {
+            _compute.Run(action).Get();
+        }
+
+        /** <inheritDoc /> */
+        public void AffinityRun(string cacheName, object affinityKey, IComputeAction action)
+        {
+            _compute.AffinityRun(cacheName, affinityKey, action).Get();
+        }
+
+        /** <inheritDoc /> */
+        public void Run(IEnumerable<IComputeAction> actions)
+        {
+            _compute.Run(actions).Get();
+        }
+
+        /** <inheritDoc /> */
+        public TR Apply<T, TR>(IComputeFunc<T, TR> clo, T arg)
+        {
+            return _compute.Apply(clo, arg).Get();
+        }
+
+        /** <inheritDoc /> */
+        public ICollection<TR> Apply<T, TR>(IComputeFunc<T, TR> clo, IEnumerable<T> args)
+        {
+            return _compute.Apply(clo, args).Get();
+        }
+
+        /** <inheritDoc /> */
+        public TR2 Apply<T, TR1, TR2>(IComputeFunc<T, TR1> clo, IEnumerable<T> args, IComputeReducer<TR1, TR2> rdc)
+        {
+            return _compute.Apply(clo, args, rdc).Get();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeAsync.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeAsync.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeAsync.cs
new file mode 100644
index 0000000..199afc2
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeAsync.cs
@@ -0,0 +1,261 @@
+/*
+ * 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.Compute
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Threading;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Compute;
+
+    /// <summary>
+    /// Asynchronous Compute facade.
+    /// </summary>
+    [SuppressMessage("Microsoft.Design", "CA1001:TypesThatOwnDisposableFieldsShouldBeDisposable")]
+    internal class ComputeAsync : ICompute
+    {
+        /** */
+        protected readonly ComputeImpl Compute;
+
+        /** Current future. */
+        private readonly ThreadLocal<IFuture> _curFut = new ThreadLocal<IFuture>();
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeAsync"/> class.
+        /// </summary>
+        /// <param name="computeImpl">The compute implementation.</param>
+        internal ComputeAsync(ComputeImpl computeImpl)
+        {
+            Compute = computeImpl;
+        }
+
+        /** <inheritDoc /> */
+        public ICompute WithAsync()
+        {
+            return this;
+        }
+
+        /** <inheritDoc /> */
+        public bool IsAsync
+        {
+            get { return true; }
+        }
+
+        /** <inheritDoc /> */
+        public IFuture GetFuture()
+        {
+            return GetFuture<object>();
+        }
+
+        /** <inheritDoc /> */
+        public IFuture<TResult> GetFuture<TResult>()
+        {
+            var fut = _curFut.Value;
+
+            if (fut == null)
+                throw new InvalidOperationException("Asynchronous operation not started.");
+
+            var fut0 = fut as IFuture<TResult>;
+
+            if (fut0 == null)
+                throw new InvalidOperationException(
+                    string.Format("Requested future type {0} is incompatible with current future type {1}",
+                        typeof(IFuture<TResult>), fut.GetType()));
+
+            _curFut.Value = null;
+
+            return fut0;
+        }
+
+        /** <inheritDoc /> */
+        public IClusterGroup ClusterGroup
+        {
+            get { return Compute.ClusterGroup; }
+        }
+
+        /** <inheritDoc /> */
+        public ICompute WithNoFailover()
+        {
+            Compute.WithNoFailover();
+
+            return this;
+        }
+
+        /** <inheritDoc /> */
+        public ICompute WithTimeout(long timeout)
+        {
+            Compute.WithTimeout(timeout);
+
+            return this;
+        }
+
+        /** <inheritDoc /> */
+        public ICompute WithKeepPortable()
+        {
+            Compute.WithKeepPortable();
+
+            return this;
+        }
+        
+        /** <inheritDoc /> */
+        public T ExecuteJavaTask<T>(string taskName, object taskArg)
+        {
+            _curFut.Value = Compute.ExecuteJavaTaskAsync<T>(taskName, taskArg);
+
+            return default(T);
+        }
+
+        /** <inheritDoc /> */
+        public TR Execute<TA, T, TR>(IComputeTask<TA, T, TR> task, TA taskArg)
+        {
+            _curFut.Value = Compute.Execute(task, taskArg);
+
+            return default(TR);
+        }
+
+        /** <inheritDoc /> */
+        public TR Execute<T, TR>(IComputeTask<T, TR> task)
+        {
+            _curFut.Value = Compute.Execute(task, null);
+
+            return default(TR);
+        }
+
+        /** <inheritDoc /> */
+        public TR Execute<TA, T, TR>(Type taskType, TA taskArg)
+        {
+            _curFut.Value = Compute.Execute<TA, T, TR>(taskType, taskArg);
+
+            return default(TR);
+        }
+
+        /** <inheritDoc /> */
+        public TR Execute<T, TR>(Type taskType)
+        {
+            _curFut.Value = Compute.Execute<object, T, TR>(taskType, null);
+
+            return default(TR);
+        }
+
+        /** <inheritDoc /> */
+        public TR Call<TR>(IComputeFunc<TR> clo)
+        {
+            _curFut.Value = Compute.Execute(clo);
+
+            return default(TR);
+        }
+
+        /** <inheritDoc /> */
+        public TR AffinityCall<TR>(string cacheName, object affinityKey, IComputeFunc<TR> clo)
+        {
+            Compute.AffinityCall(cacheName, affinityKey, clo);
+
+            return default(TR);
+        }
+
+        /** <inheritDoc /> */
+        public TR Call<TR>(Func<TR> func)
+        {
+            _curFut.Value = Compute.Execute(func);
+
+            return default(TR);
+        }
+
+        /** <inheritDoc /> */
+        public ICollection<TR> Call<TR>(IEnumerable<IComputeFunc<TR>> clos)
+        {
+            _curFut.Value = Compute.Execute(clos);
+
+            return null;
+        }
+
+        /** <inheritDoc /> */
+        public TR2 Call<TR1, TR2>(IEnumerable<IComputeFunc<TR1>> clos, IComputeReducer<TR1, TR2> rdc)
+        {
+            _curFut.Value = Compute.Execute(clos, rdc);
+
+            return default(TR2);
+        }
+
+        /** <inheritDoc /> */
+        public ICollection<TR> Broadcast<TR>(IComputeFunc<TR> clo)
+        {
+            _curFut.Value = Compute.Broadcast(clo);
+
+            return null;
+        }
+
+        /** <inheritDoc /> */
+        public ICollection<TR> Broadcast<T, TR>(IComputeFunc<T, TR> clo, T arg)
+        {
+            _curFut.Value = Compute.Broadcast(clo, arg);
+
+            return null;
+        }
+
+        /** <inheritDoc /> */
+        public void Broadcast(IComputeAction action)
+        {
+            _curFut.Value = Compute.Broadcast(action);
+        }
+
+        /** <inheritDoc /> */
+        public void Run(IComputeAction action)
+        {
+            _curFut.Value = Compute.Run(action);
+        }
+
+        /** <inheritDoc /> */
+        public void AffinityRun(string cacheName, object affinityKey, IComputeAction action)
+        {
+            Compute.AffinityRun(cacheName, affinityKey, action);
+        }
+
+        /** <inheritDoc /> */
+        public void Run(IEnumerable<IComputeAction> actions)
+        {
+            _curFut.Value = Compute.Run(actions);
+        }
+
+        /** <inheritDoc /> */
+        public TR Apply<T, TR>(IComputeFunc<T, TR> clo, T arg)
+        {
+            _curFut.Value = Compute.Apply(clo, arg);
+
+            return default(TR);
+        }
+
+        /** <inheritDoc /> */
+        public ICollection<TR> Apply<T, TR>(IComputeFunc<T, TR> clo, IEnumerable<T> args)
+        {
+            _curFut.Value = Compute.Apply(clo, args);
+
+            return null;
+        }
+
+        /** <inheritDoc /> */
+        public TR2 Apply<T, TR1, TR2>(IComputeFunc<T, TR1> clo, IEnumerable<T> args, IComputeReducer<TR1, TR2> rdc)
+        {
+            _curFut.Value = Compute.Apply(clo, args, rdc);
+
+            return default(TR2);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeFunc.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeFunc.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeFunc.cs
new file mode 100644
index 0000000..a971418
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeFunc.cs
@@ -0,0 +1,119 @@
+/*
+ * 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.Compute
+{
+    using System;
+    using System.Reflection;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Resource;
+    using Apache.Ignite.Core.Portable;
+    using Apache.Ignite.Core.Resource;
+
+    /// <summary>
+    /// Non-generic version of IComputeFunc{T}.
+    /// </summary>
+    internal interface IComputeFunc : IComputeFunc<object, object>
+    {
+        // No-op
+    }
+
+    /// <summary>
+    /// Wraps generic func into a non-generic for internal usage.
+    /// </summary>
+    internal class ComputeFuncWrapper : IComputeFunc, IPortableWriteAware
+    {
+        /** */
+        private readonly object _func;
+
+        /** */
+        private readonly Func<object, object, object> _invoker;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeFuncWrapper" /> class.
+        /// </summary>
+        /// <param name="func">The function to wrap.</param>
+        /// <param name="invoker">The function invoker.</param>
+        public ComputeFuncWrapper(object func, Func<object, object> invoker)
+        {
+            _func = func;
+
+            _invoker = (target, arg) => invoker(arg);
+        }
+
+        /** <inheritDoc /> */
+        public object Invoke(object arg)
+        {
+            try
+            {
+                return _invoker(_func, arg);
+            }
+            catch (TargetInvocationException ex)
+            {
+                throw ex.InnerException;
+            }
+        }
+
+        /** <inheritDoc /> */
+        public void WritePortable(IPortableWriter writer)
+        {
+            var writer0 = (PortableWriterImpl)writer.RawWriter();
+
+            writer0.DetachNext();
+            PortableUtils.WritePortableOrSerializable(writer0, _func);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeFuncWrapper"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        public ComputeFuncWrapper(IPortableReader reader)
+        {
+            var reader0 = (PortableReaderImpl)reader.RawReader();
+
+            _func = PortableUtils.ReadPortableOrSerializable<object>(reader0);
+
+            _invoker = DelegateTypeDescriptor.GetComputeFunc(_func.GetType());
+        }
+
+        /// <summary>
+        /// Injects the Ignite instance.
+        /// </summary>
+        [InstanceResource]
+        public void InjectIgnite(IIgnite ignite)
+        {
+            // Propagate injection
+            ResourceProcessor.Inject(_func, (IgniteProxy) ignite);
+        }
+    }    
+    
+    /// <summary>
+    /// Extension methods for IComputeFunc{T}.
+    /// </summary>
+    internal static class ComputeFuncExtensions
+    {
+        /// <summary>
+        /// Convert to non-generic wrapper.
+        /// </summary>
+        public static IComputeFunc ToNonGeneric<T, TR>(this IComputeFunc<T, TR> func)
+        {
+            return new ComputeFuncWrapper(func, x => func.Invoke((T) x));
+        }
+    }
+}


[21/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableUtils.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableUtils.cs
new file mode 100644
index 0000000..3143e45
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableUtils.cs
@@ -0,0 +1,2039 @@
+/*
+ * 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.Portable
+{
+    using System;
+    using System.Collections;
+    using System.Collections.Concurrent;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using System.IO;
+    using System.Reflection;
+    using System.Runtime.Serialization.Formatters.Binary;
+    using System.Text;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Portable;
+
+    /**
+     * <summary>Utilities for portable serialization.</summary>
+     */
+    static class PortableUtils
+    {
+        /** Cache empty dictionary. */
+        public static readonly IDictionary<int, int> EmptyFields = new Dictionary<int, int>();
+
+        /** Header of NULL object. */
+        public const byte HdrNull = 101;
+
+        /** Header of object handle. */
+        public const byte HdrHnd = 102;
+
+        /** Header of object in fully serialized form. */
+        public const byte HdrFull = 103;
+        
+        /** Full header length. */
+        public const int FullHdrLen = 18;
+
+        /** Type: object. */
+        public const byte TypeObject = HdrFull;
+
+        /** Type: unsigned byte. */
+        public const byte TypeByte = 1;
+
+        /** Type: short. */
+        public const byte TypeShort = 2;
+
+        /** Type: int. */
+        public const byte TypeInt = 3;
+
+        /** Type: long. */
+        public const byte TypeLong = 4;
+
+        /** Type: float. */
+        public const byte TypeFloat = 5;
+
+        /** Type: double. */
+        public const byte TypeDouble = 6;
+
+        /** Type: char. */
+        public const byte TypeChar = 7;
+
+        /** Type: boolean. */
+        public const byte TypeBool = 8;
+        
+        /** Type: decimal. */
+        public const byte TypeDecimal = 30;
+
+        /** Type: string. */
+        public const byte TypeString = 9;
+
+        /** Type: GUID. */
+        public const byte TypeGuid = 10;
+
+        /** Type: date. */
+        public const byte TypeDate = 11;
+
+        /** Type: unsigned byte array. */
+        public const byte TypeArrayByte = 12;
+
+        /** Type: short array. */
+        public const byte TypeArrayShort = 13;
+
+        /** Type: int array. */
+        public const byte TypeArrayInt = 14;
+
+        /** Type: long array. */
+        public const byte TypeArrayLong = 15;
+
+        /** Type: float array. */
+        public const byte TypeArrayFloat = 16;
+
+        /** Type: double array. */
+        public const byte TypeArrayDouble = 17;
+
+        /** Type: char array. */
+        public const byte TypeArrayChar = 18;
+
+        /** Type: boolean array. */
+        public const byte TypeArrayBool = 19;
+
+        /** Type: decimal array. */
+        public const byte TypeArrayDecimal = 31;
+
+        /** Type: string array. */
+        public const byte TypeArrayString = 20;
+
+        /** Type: GUID array. */
+        public const byte TypeArrayGuid = 21;
+
+        /** Type: date array. */
+        public const byte TypeArrayDate = 22;
+
+        /** Type: object array. */
+        public const byte TypeArray = 23;
+
+        /** Type: collection. */
+        public const byte TypeCollection = 24;
+
+        /** Type: map. */
+        public const byte TypeDictionary = 25;
+
+        /** Type: map entry. */
+        public const byte TypeMapEntry = 26;
+
+        /** Type: portable object. */
+        public const byte TypePortable = 27;
+
+        /** Type: enum. */
+        public const byte TypeEnum = 28;
+
+        /** Type: enum array. */
+        public const byte TypeArrayEnum = 29;
+        
+        /** Type: native job holder. */
+        public const byte TypeNativeJobHolder = 77;
+
+        /** Type: native job result holder. */
+        public const byte TypePortableJobResHolder = 76;
+
+        /** Type: .Net configuration. */
+        public const byte TypeDotNetCfg = 202;
+
+        /** Type: .Net portable configuration. */
+        public const byte TypeDotNetPortableCfg = 203;
+
+        /** Type: .Net portable type configuration. */
+        public const byte TypeDotNetPortableTypCfg = 204;
+
+        /** Type: Ignite proxy. */
+        public const byte TypeIgniteProxy = 74;
+
+        /** Type: function wrapper. */
+        public const byte TypeComputeOutFuncJob = 80;
+
+        /** Type: function wrapper. */
+        public const byte TypeComputeFuncJob = 81;
+
+        /** Type: continuous query remote filter. */
+        public const byte TypeContinuousQueryRemoteFilterHolder = 82;
+
+        /** Type: Compute out func wrapper. */
+        public const byte TypeComputeOutFuncWrapper = 83;
+
+        /** Type: Compute func wrapper. */
+        public const byte TypeComputeFuncWrapper = 85;
+
+        /** Type: Compute job wrapper. */
+        public const byte TypeComputeJobWrapper = 86;
+
+        /** Type: Compute job wrapper. */
+        public const byte TypeSerializableHolder = 87;
+
+        /** Type: action wrapper. */
+        public const byte TypeComputeActionJob = 88;
+
+        /** Type: entry processor holder. */
+        public const byte TypeCacheEntryProcessorHolder = 89;
+
+        /** Type: entry predicate holder. */
+        public const byte TypeCacheEntryPredicateHolder = 90;
+        
+        /** Type: product license. */
+        public const byte TypeProductLicense = 78;
+
+        /** Type: message filter holder. */
+        public const byte TypeMessageFilterHolder = 92;
+
+        /** Type: message filter holder. */
+        public const byte TypePortableOrSerializableHolder = 93;
+
+        /** Type: stream receiver holder. */
+        public const byte TypeStreamReceiverHolder = 94;
+
+        /** Collection: custom. */
+        public const byte CollectionCustom = 0;
+
+        /** Collection: array list. */
+        public const byte CollectionArrayList = 1;
+
+        /** Collection: linked list. */
+        public const byte CollectionLinkedList = 2;
+
+        /** Collection: hash set. */
+        public const byte CollectionHashSet = 3;
+
+        /** Collection: hash set. */
+        public const byte CollectionLinkedHashSet = 4;
+
+        /** Collection: sorted set. */
+        public const byte CollectionSortedSet = 5;
+
+        /** Collection: concurrent bag. */
+        public const byte CollectionConcurrentBag = 6;
+
+        /** Map: custom. */
+        public const byte MapCustom = 0;
+
+        /** Map: hash map. */
+        public const byte MapHashMap = 1;
+
+        /** Map: linked hash map. */
+        public const byte MapLinkedHashMap = 2;
+
+        /** Map: sorted map. */
+        public const byte MapSortedMap = 3;
+
+        /** Map: concurrent hash map. */
+        public const byte MapConcurrentHashMap = 4;
+
+        /** Byte "0". */
+        public const byte ByteZero = 0;
+
+        /** Byte "1". */
+        public const byte ByteOne = 1;
+
+        /** Indicates object array. */
+        public const int ObjTypeId = -1;
+
+        /** Int type. */
+        public static readonly Type TypInt = typeof(int);
+
+        /** Collection type. */
+        public static readonly Type TypCollection = typeof(ICollection);
+
+        /** Dictionary type. */
+        public static readonly Type TypDictionary = typeof(IDictionary);
+
+        /** Generic collection type. */
+        public static readonly Type TypGenericCollection = typeof(ICollection<>);
+
+        /** Generic dictionary type. */
+        public static readonly Type TypGenericDictionary = typeof(IDictionary<,>);
+
+        /** Ticks for Java epoch. */
+        private static readonly long JavaDateTicks = new DateTime(1970, 1, 1, 0, 0, 0, 0, DateTimeKind.Utc).Ticks;
+        
+        /** Bindig flags for static search. */
+        private static BindingFlags _bindFlagsStatic = 
+            BindingFlags.Static | BindingFlags.Public | BindingFlags.NonPublic;
+
+        /** Default poratble marshaller. */
+        private static readonly PortableMarshaller Marsh = new PortableMarshaller(null);
+
+        /** Method: WriteGenericCollection. */
+        public static readonly MethodInfo MtdhWriteGenericCollection =
+            typeof(PortableUtils).GetMethod("WriteGenericCollection", _bindFlagsStatic);
+
+        /** Method: ReadGenericCollection. */
+        public static readonly MethodInfo MtdhReadGenericCollection =
+            typeof(PortableUtils).GetMethod("ReadGenericCollection", _bindFlagsStatic);
+
+        /** Method: WriteGenericDictionary. */
+        public static readonly MethodInfo MtdhWriteGenericDictionary =
+            typeof(PortableUtils).GetMethod("WriteGenericDictionary", _bindFlagsStatic);
+
+        /** Method: ReadGenericDictionary. */
+        public static readonly MethodInfo MtdhReadGenericDictionary =
+            typeof(PortableUtils).GetMethod("ReadGenericDictionary", _bindFlagsStatic);
+
+        /** Method: ReadGenericArray. */
+        public static readonly MethodInfo MtdhReadGenericArray =
+            typeof(PortableUtils).GetMethod("ReadGenericArray", _bindFlagsStatic);
+
+        /** Cached UTF8 encoding. */
+        private static readonly Encoding Utf8 = Encoding.UTF8;
+
+        /** Cached generic array read funcs. */
+        private static readonly CopyOnWriteConcurrentDictionary<Type, Func<PortableReaderImpl, bool, object>>
+            ArrayReaders = new CopyOnWriteConcurrentDictionary<Type, Func<PortableReaderImpl, bool, object>>();
+
+        /// <summary>
+        /// Default marshaller.
+        /// </summary>
+        public static PortableMarshaller Marshaller
+        {
+            get { return Marsh; }
+        }
+
+        /**
+         * <summary>Write boolean array.</summary>
+         * <param name="vals">Value.</param>
+         * <param name="stream">Output stream.</param>
+         */
+        public static void WriteBooleanArray(bool[] vals, IPortableStream stream)
+        {
+            stream.WriteInt(vals.Length);
+
+            stream.WriteBoolArray(vals);
+        }
+
+        /**
+         * <summary>Read boolean array.</summary>
+         * <param name="stream">Output stream.</param>
+         * <returns>Value.</returns>
+         */
+        public static bool[] ReadBooleanArray(IPortableStream stream)
+        {
+            int len = stream.ReadInt();
+
+            return stream.ReadBoolArray(len);
+        }
+
+        /**
+         * <summary>Write byte array.</summary>
+         * <param name="vals">Value.</param>
+         * <param name="stream">Output stream.</param>
+         * <returns>Length of written data.</returns>
+         */
+        public static void WriteByteArray(byte[] vals, IPortableStream stream)
+        {
+            stream.WriteInt(vals.Length);
+
+            stream.WriteByteArray(vals);
+        }
+
+        /**
+         * <summary>Read byte array.</summary>
+         * <param name="stream">Output stream.</param>
+         * <returns>Value.</returns>
+         */
+        public static byte[] ReadByteArray(IPortableStream stream)
+        {
+            return stream.ReadByteArray(stream.ReadInt());
+        }
+
+        /**
+         * <summary>Read byte array.</summary>
+         * <param name="stream">Output stream.</param>
+         * <returns>Value.</returns>
+         */
+        public static unsafe sbyte[] ReadSbyteArray(IPortableStream stream)
+        {
+            int len = stream.ReadInt();
+
+            sbyte[] res = new sbyte[len];
+
+            fixed (sbyte* res0 = res)
+            {
+                stream.Read((byte*) res0, len);
+            }
+
+            return res;
+        }
+
+        /**
+         * <summary>Read byte array.</summary>
+         * <param name="data">Data.</param>
+         * <param name="pos">Position.</param>
+         * <returns>Value.</returns>
+         */
+        public static byte[] ReadByteArray(byte[] data, int pos) {
+            int len = ReadInt(data, pos);
+
+            pos += 4;
+
+            byte[] res = new byte[len];
+
+            Buffer.BlockCopy(data, pos, res, 0, len);
+
+            return res;
+        }
+
+        /**
+         * <summary>Write short array.</summary>
+         * <param name="vals">Value.</param>
+         * <param name="stream">Output stream.</param>
+         */
+        public static void WriteShortArray(short[] vals, IPortableStream stream)
+        {
+            stream.WriteInt(vals.Length);
+
+            stream.WriteShortArray(vals);
+        }
+
+        /**
+         * <summary>Read short array.</summary>
+         * <param name="stream">Stream.</param>
+         * <returns>Value.</returns>
+         */
+        public static unsafe ushort[] ReadUshortArray(IPortableStream stream)
+        {
+            int len = stream.ReadInt();
+
+            ushort[] res = new ushort[len];
+
+            fixed (ushort* res0 = res)
+            {
+                stream.Read((byte*) res0, len * 2);
+            }
+
+            return res;
+        }
+
+        /**
+         * <summary>Read short array.</summary>
+         * <param name="stream">Stream.</param>
+         * <returns>Value.</returns>
+         */
+        public static short[] ReadShortArray(IPortableStream stream)
+        {
+            return stream.ReadShortArray(stream.ReadInt());
+        }
+
+        /**
+         * <summary>Read int value.</summary>
+         * <param name="data">Data array.</param>
+         * <param name="pos">Position.</param>
+         * <returns>Value.</returns>
+         */
+        public static int ReadInt(byte[] data, int pos) {
+            int val = data[pos];
+
+            val |= data[pos + 1] << 8;
+            val |= data[pos + 2] << 16;
+            val |= data[pos + 3] << 24;
+
+            return val;
+        }
+
+        /**
+         * <summary>Read long value.</summary>
+         * <param name="data">Data array.</param>
+         * <param name="pos">Position.</param>
+         * <returns>Value.</returns>
+         */
+        public static long ReadLong(byte[] data, int pos) {
+            long val = (long)(data[pos]) << 0;
+
+            val |= (long)(data[pos + 1]) << 8;
+            val |= (long)(data[pos + 2]) << 16;
+            val |= (long)(data[pos + 3]) << 24;
+            val |= (long)(data[pos + 4]) << 32;
+            val |= (long)(data[pos + 5]) << 40;
+            val |= (long)(data[pos + 6]) << 48;
+            val |= (long)(data[pos + 7]) << 56;
+
+            return val;
+        }
+
+        /**
+         * <summary>Write int array.</summary>
+         * <param name="vals">Value.</param>
+         * <param name="stream">Output stream.</param>
+         */
+        public static void WriteIntArray(int[] vals, IPortableStream stream)
+        {
+            stream.WriteInt(vals.Length);
+
+            stream.WriteIntArray(vals);
+        }
+
+        /**
+         * <summary>Read int array.</summary>
+         * <param name="stream">Stream.</param>
+         * <returns>Value.</returns>
+         */
+        public static int[] ReadIntArray(IPortableStream stream)
+        {
+            return stream.ReadIntArray(stream.ReadInt());
+        }
+
+        /**
+         * <summary>Read int array.</summary>
+         * <param name="stream">Stream.</param>
+         * <returns>Value.</returns>
+         */
+        public static unsafe uint[] ReadUintArray(IPortableStream stream)
+        {
+            int len = stream.ReadInt();
+
+            uint[] res = new uint[len];
+
+            fixed (uint* res0 = res)
+            {
+                stream.Read((byte*) res0, len * 4);
+            }
+
+            return res;
+        }
+
+        /**
+         * <summary>Write long array.</summary>
+         * <param name="vals">Value.</param>
+         * <param name="stream">Output stream.</param>
+         */
+        public static void WriteLongArray(long[] vals, IPortableStream stream)
+        {
+            stream.WriteInt(vals.Length);
+
+            stream.WriteLongArray(vals);
+        }
+
+        /**
+         * <summary>Read long array.</summary>
+         * <param name="stream">Stream.</param>
+         * <returns>Value.</returns>
+         */
+        public static long[] ReadLongArray(IPortableStream stream)
+        {
+            return stream.ReadLongArray(stream.ReadInt());
+        }
+
+        /**
+         * <summary>Read ulong array.</summary>
+         * <param name="stream">Stream.</param>
+         * <returns>Value.</returns>
+         */
+        public static unsafe ulong[] ReadUlongArray(IPortableStream stream)
+        {
+            int len = stream.ReadInt();
+
+            ulong[] res = new ulong[len];
+
+            fixed (ulong* res0 = res)
+            {
+                stream.Read((byte*) res0, len * 8);
+            }
+
+            return res;
+        }
+
+        /**
+         * <summary>Write char array.</summary>
+         * <param name="vals">Value.</param>
+         * <param name="stream">Output stream.</param>
+         */
+        public static void WriteCharArray(char[] vals, IPortableStream stream)
+        {
+            stream.WriteInt(vals.Length);
+
+            stream.WriteCharArray(vals);
+        }
+
+        /**
+         * <summary>Read char array.</summary>
+         * <param name="stream">Stream.</param>
+         * <returns>Value.</returns>
+         */
+        public static char[] ReadCharArray(IPortableStream stream)
+        {
+            int len = stream.ReadInt();
+
+            return stream.ReadCharArray(len);
+        }
+
+        /**
+         * <summary>Write float array.</summary>
+         * <param name="vals">Value.</param>
+         * <param name="stream">Output stream.</param>
+         */
+        public static void WriteFloatArray(float[] vals, IPortableStream stream)
+        {
+            stream.WriteInt(vals.Length);
+
+            stream.WriteFloatArray(vals);
+        }
+
+        /**
+         * <summary>Read float array.</summary>
+         * <param name="stream">Stream.</param>
+         * <returns>Value.</returns>
+         */
+        public static float[] ReadFloatArray(IPortableStream stream)
+        {
+            int len = stream.ReadInt();
+
+            return stream.ReadFloatArray(len);
+        }
+
+        /**
+         * <summary>Write double array.</summary>
+         * <param name="vals">Value.</param>
+         * <param name="stream">Output stream.</param>
+         */
+        public static void WriteDoubleArray(double[] vals, IPortableStream stream)
+        {
+            stream.WriteInt(vals.Length);
+
+            stream.WriteDoubleArray(vals);
+        }
+
+        /**
+         * <summary>Read double array.</summary>
+         * <param name="stream">Stream.</param>
+         * <returns>Value.</returns>
+         */
+        public static double[] ReadDoubleArray(IPortableStream stream)
+        {
+            int len = stream.ReadInt();
+
+            return stream.ReadDoubleArray(len);
+        }
+
+        /**
+         * <summary>Write date.</summary>
+         * <param name="val">Date.</param>
+         * <param name="stream">Stream.</param>
+         */
+        public static void WriteDate(DateTime? val, IPortableStream stream)
+        {
+            long high;
+            int low;
+
+            Debug.Assert(val.HasValue);
+            ToJavaDate(val.Value, out high, out low);
+
+            stream.WriteLong(high);
+            stream.WriteInt(low);
+        }
+
+        /**
+         * <summary>Read date.</summary>
+         * <param name="stream">Stream.</param>
+         * <param name="local">Local flag.</param>
+         * <returns>Date</returns>
+         */
+        public static DateTime? ReadDate(IPortableStream stream, bool local)
+        {
+            long high = stream.ReadLong();
+            int low = stream.ReadInt();
+
+            return ToDotNetDate(high, low, local);
+        }
+
+        /**
+         * <summary>Write date array.</summary>
+         * <param name="vals">Date array.</param>
+         * <param name="stream">Stream.</param>
+         */
+        public static void WriteDateArray(DateTime?[] vals, IPortableStream stream)
+        {
+            stream.WriteInt(vals.Length);
+
+            foreach (DateTime? val in vals)
+            {
+                if (val.HasValue)
+                    PortableSystemHandlers.WriteHndDateTyped(stream, val);
+                else
+                    stream.WriteByte(HdrNull);
+            }
+        }
+        
+        /**
+         * <summary>Write string in UTF8 encoding.</summary>
+         * <param name="val">String.</param>
+         * <param name="stream">Stream.</param>
+         */
+        public static unsafe void WriteString(string val, IPortableStream stream)
+        {
+            stream.WriteBool(true);
+
+            int charCnt = val.Length;
+
+            fixed (char* chars = val)
+            {
+                int byteCnt = Utf8.GetByteCount(chars, charCnt);
+
+                stream.WriteInt(byteCnt);
+
+                stream.WriteString(chars, charCnt, byteCnt, Utf8);
+            }
+        }
+
+        /**
+         * <summary>Read string in UTF8 encoding.</summary>
+         * <param name="stream">Stream.</param>
+         * <returns>String.</returns>
+         */
+        public static string ReadString(IPortableStream stream)
+        {
+            if (stream.ReadBool())
+            {
+                byte[] bytes = ReadByteArray(stream);
+
+                return bytes != null ? Utf8.GetString(bytes) : null;
+            }
+            
+            char[] chars = ReadCharArray(stream);
+
+            return new string(chars);
+        }
+
+        /**
+         * <summary>Write string array in UTF8 encoding.</summary>
+         * <param name="vals">String array.</param>
+         * <param name="stream">Stream.</param>
+         */
+        public static void WriteStringArray(string[] vals, IPortableStream stream)
+        {
+            stream.WriteInt(vals.Length);
+
+            foreach (string val in vals)
+            {
+                if (val != null)
+                    PortableSystemHandlers.WriteHndStringTyped(stream, val); 
+                else
+                    stream.WriteByte(HdrNull);
+            }
+        }
+
+        /**
+         * <summary>Read string array in UTF8 encoding.</summary>
+         * <param name="stream">Stream.</param>
+         * <returns>String array.</returns>
+         */
+        public static string[] ReadStringArray(IPortableStream stream)
+        {
+            int len = stream.ReadInt();
+
+            string[] vals = new string[len];
+
+            for (int i = 0; i < len; i++)
+                vals[i] = ReadString(stream);
+
+            return vals;
+        }
+
+        /**
+         * <summary>Write decimal value.</summary>
+         * <param name="val">Decimal value.</param>
+         * <param name="stream">Stream.</param>
+         */
+        public static void WriteDecimal(decimal val, IPortableStream stream) 
+        {
+            // Vals are:
+            // [0] = lo
+            // [1] = mid
+            // [2] = high
+            // [3] = flags
+            int[] vals = decimal.GetBits(val);
+            
+            // Get start index skipping leading zeros.
+            int idx = vals[2] != 0 ? 2 : vals[1] != 0 ? 1 : vals[0] != 0 ? 0 : -1;
+                        
+            // Write scale and negative flag.
+            int scale = (vals[3] & 0x00FF0000) >> 16; 
+
+            stream.WriteInt(((vals[3] & 0x80000000) == 0x80000000) ? (int)((uint)scale | 0x80000000) : scale);
+
+            if (idx == -1)
+            {
+                // Writing zero.
+                stream.WriteInt(1);
+                stream.WriteByte(0);
+            }
+            else
+            {
+                int len = (idx + 1) << 2;
+                
+                // Write data.
+                for (int i = idx; i >= 0; i--)
+                {
+                    int curPart = vals[i];
+
+                    int part24 = (curPart >> 24) & 0xFF;
+                    int part16 = (curPart >> 16) & 0xFF;
+                    int part8 = (curPart >> 8) & 0xFF;
+                    int part0 = curPart & 0xFF;
+                    
+                    if (i == idx)
+                    {
+                        // Possibly skipping some values here.
+                        if (part24 != 0)
+                        {
+                            if ((part24 & 0x80) == 0x80)
+                            {
+                                stream.WriteInt(len + 1);
+
+                                stream.WriteByte(ByteZero);
+                            }
+                            else
+                                stream.WriteInt(len);
+
+                            stream.WriteByte((byte)part24);
+                            stream.WriteByte((byte)part16);
+                            stream.WriteByte((byte)part8);
+                            stream.WriteByte((byte)part0);
+                        }
+                        else if (part16 != 0)
+                        {
+                            if ((part16 & 0x80) == 0x80)
+                            {
+                                stream.WriteInt(len);
+
+                                stream.WriteByte(ByteZero);
+                            }
+                            else
+                                stream.WriteInt(len - 1);
+
+                            stream.WriteByte((byte)part16);
+                            stream.WriteByte((byte)part8);
+                            stream.WriteByte((byte)part0);
+                        }
+                        else if (part8 != 0)
+                        {
+                            if ((part8 & 0x80) == 0x80)
+                            {
+                                stream.WriteInt(len - 1);
+
+                                stream.WriteByte(ByteZero);
+                            }
+                            else
+                                stream.WriteInt(len - 2);
+
+                            stream.WriteByte((byte)part8);
+                            stream.WriteByte((byte)part0);
+                        }
+                        else
+                        {
+                            if ((part0 & 0x80) == 0x80)
+                            {
+                                stream.WriteInt(len - 2);
+
+                                stream.WriteByte(ByteZero);
+                            }
+                            else
+                                stream.WriteInt(len - 3);
+
+                            stream.WriteByte((byte)part0);
+                        }
+                    }
+                    else
+                    {
+                        stream.WriteByte((byte)part24);
+                        stream.WriteByte((byte)part16);
+                        stream.WriteByte((byte)part8);
+                        stream.WriteByte((byte)part0);
+                    }
+                }
+            }
+        }
+
+        /**
+         * <summary>Read decimal value.</summary>
+         * <param name="stream">Stream.</param>
+         * <returns>Decimal value.</returns>
+         */
+        public static decimal ReadDecimal(IPortableStream stream)
+        {
+            int scale = stream.ReadInt();
+
+            bool neg;
+
+            if (scale < 0)
+            {
+                scale = scale & 0x7FFFFFFF;
+
+                neg = true;
+            }
+            else
+                neg = false;
+
+            byte[] mag = ReadByteArray(stream);
+
+            if (scale < 0 || scale > 28)
+                throw new PortableException("Decimal value scale overflow (must be between 0 and 28): " + scale);
+
+            if (mag.Length > 13)
+                throw new PortableException("Decimal magnitude overflow (must be less than 96 bits): " + 
+                    mag.Length * 8);
+
+            if (mag.Length == 13 && mag[0] != 0)
+                throw new PortableException("Decimal magnitude overflow (must be less than 96 bits): " +
+                        mag.Length * 8);
+
+            int hi = 0;
+            int mid = 0;
+            int lo = 0;
+
+            int ctr = -1;
+
+            for (int i = mag.Length - 12; i < mag.Length; i++)
+            {
+                if (++ctr == 4)
+                {
+                    mid = lo;
+                    lo = 0;
+                }
+                else if (ctr == 8)
+                {
+                    hi = mid;
+                    mid = lo;
+                    lo = 0;
+                }
+
+                if (i >= 0)
+                    lo = (lo << 8) + mag[i];
+            }
+
+            return new decimal(lo, mid, hi, neg, (byte)scale);
+        }
+
+        /**
+         * <summary>Write decimal array.</summary>
+         * <param name="vals">Decimal array.</param>
+         * <param name="stream">Stream.</param>
+         */
+        public static void WriteDecimalArray(decimal[] vals, IPortableStream stream)
+        {
+            stream.WriteInt(vals.Length);
+
+            foreach (decimal val in vals)
+                WriteDecimal(val, stream);
+        }
+
+        /**
+         * <summary>Read decimal array.</summary>
+         * <param name="stream">Stream.</param>
+         * <returns>Decimal array.</returns>
+         */
+        public static decimal[] ReadDecimalArray(IPortableStream stream)
+        {
+            int len = stream.ReadInt();
+
+            decimal[] vals = new decimal[len];
+
+            for (int i = 0; i < len; i++)
+                vals[i] = ReadDecimal(stream);
+
+            return vals;
+        }
+
+        /**
+         * <summary>Write GUID.</summary>
+         * <param name="val">GUID.</param>
+         * <param name="stream">Stream.</param>
+         */
+        public static unsafe void WriteGuid(Guid? val, IPortableStream stream)
+        {
+            Debug.Assert(val.HasValue);
+            byte[] bytes = val.Value.ToByteArray();
+
+            // .Net returns bytes in the following order: _a(4), _b(2), _c(2), _d, _e, _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.
+            fixed (byte* bytes0 = bytes)
+            {
+                stream.Write(bytes0 + 6, 2); // _c
+                stream.Write(bytes0 + 4, 2); // _a
+                stream.Write(bytes0, 4);     // _a
+            }
+
+            stream.WriteByte(bytes[15]); // _k
+            stream.WriteByte(bytes[14]); // _j
+            stream.WriteByte(bytes[13]); // _i
+            stream.WriteByte(bytes[12]); // _h
+
+            stream.WriteByte(bytes[11]); // _g
+            stream.WriteByte(bytes[10]); // _f
+            stream.WriteByte(bytes[9]);  // _e
+            stream.WriteByte(bytes[8]);  // _d
+        }
+
+        /**
+         * <summary>Read GUID.</summary>
+         * <param name="stream">Stream.</param>
+         * <returns>GUID</returns>
+         */
+        public static unsafe Guid? ReadGuid(IPortableStream stream)
+        {
+            byte[] bytes = new byte[16];
+
+            // Perform conversion opposite to what write does.
+            fixed (byte* bytes0 = bytes)
+            {
+                stream.Read(bytes0 + 6, 2);      // _c
+                stream.Read(bytes0 + 4, 2);      // _b
+                stream.Read(bytes0, 4);          // _a
+            }
+
+            bytes[15] = stream.ReadByte();  // _k
+            bytes[14] = stream.ReadByte();  // _j
+            bytes[13] = stream.ReadByte();  // _i
+            bytes[12] = stream.ReadByte();  // _h
+
+            bytes[11] = stream.ReadByte();  // _g
+            bytes[10] = stream.ReadByte();  // _f
+            bytes[9] = stream.ReadByte();   // _e
+            bytes[8] = stream.ReadByte();   // _d
+
+            return new Guid(bytes);
+        }
+
+        /**
+         * <summary>Read GUID.</summary>
+         * <param name="data">Data array.</param>
+         * <param name="pos">Position.</param>
+         * <returns>GUID</returns>
+         */
+        public static Guid ReadGuid(byte[] data, int pos) {
+            byte[] bytes = new byte[16];
+
+            // Perform conversion opposite to what write does.
+            bytes[6] = data[pos];  // _c
+            bytes[7] = data[pos + 1];
+
+            bytes[4] = data[pos + 2];  // _b
+            bytes[5] = data[pos + 3];
+
+            bytes[0] = data[pos + 4];  // _a
+            bytes[1] = data[pos + 5];
+            bytes[2] = data[pos + 6];
+            bytes[3] = data[pos + 7];
+
+            bytes[15] = data[pos + 8];  // _k
+            bytes[14] = data[pos + 9];  // _j
+            bytes[13] = data[pos + 10];  // _i
+            bytes[12] = data[pos + 11];  // _h
+
+            bytes[11] = data[pos + 12];  // _g
+            bytes[10] = data[pos + 13];  // _f
+            bytes[9] = data[pos + 14];   // _e
+            bytes[8] = data[pos + 15];   // _d
+
+            return new Guid(bytes);
+        }
+
+        /**
+         * <summary>Write GUID array.</summary>
+         * <param name="vals">GUID array.</param>
+         * <param name="stream">Stream.</param>
+         */
+        public static void WriteGuidArray(Guid?[] vals, IPortableStream stream)
+        {
+            stream.WriteInt(vals.Length);
+
+            foreach (Guid? val in vals)
+            {
+                if (val.HasValue)
+                    PortableSystemHandlers.WriteHndGuidTyped(stream, val);
+                else
+                    stream.WriteByte(HdrNull);
+            }
+        }
+
+        /**
+         * <summary>Read GUID array.</summary>
+         * <param name="stream">Stream.</param>
+         * <returns>GUID array.</returns>
+         */
+        public static Guid?[] ReadGuidArray(IPortableStream stream)
+        {
+            int len = stream.ReadInt();
+
+            Guid?[] vals = new Guid?[len];
+
+            for (int i = 0; i < len; i++)
+                vals[i] = ReadGuid(stream);
+
+            return vals;
+        }
+        
+        /// <summary>
+        /// Write array.
+        /// </summary>
+        /// <param name="val">Array.</param>
+        /// <param name="ctx">Write context.</param>
+        /// <param name="typed">Typed flag.</param>
+        public static void WriteArray(Array val, PortableWriterImpl ctx, bool typed)
+        {
+            IPortableStream stream = ctx.Stream;
+
+            if (typed)
+                stream.WriteInt(ObjTypeId);
+
+            stream.WriteInt(val.Length);
+
+            for (int i = 0; i < val.Length; i++)
+                ctx.Write(val.GetValue(i));
+        }
+
+        /// <summary>
+        /// Read array.
+        /// </summary>
+        /// <param name="ctx">Read context.</param>
+        /// <param name="typed">Typed flag.</param>
+        /// <param name="elementType">Type of the element.</param>
+        /// <returns>Array.</returns>
+        public static object ReadArray(PortableReaderImpl ctx, bool typed, Type elementType)
+        {
+            Func<PortableReaderImpl, bool, object> result;
+
+            if (!ArrayReaders.TryGetValue(elementType, out result))
+                result = ArrayReaders.GetOrAdd(elementType, t =>
+                    DelegateConverter.CompileFunc<Func<PortableReaderImpl, bool, object>>(null,
+                        MtdhReadGenericArray.MakeGenericMethod(t),
+                        new[] {typeof (PortableReaderImpl), typeof (bool)}, new[] {false, false, true}));
+
+            return result(ctx, typed);
+        }
+
+        /// <summary>
+        /// Read array.
+        /// </summary>
+        /// <param name="ctx">Read context.</param>
+        /// <param name="typed">Typed flag.</param>
+        /// <returns>Array.</returns>
+        public static T[] ReadGenericArray<T>(PortableReaderImpl ctx, bool typed)
+        {
+            IPortableStream stream = ctx.Stream;
+
+            if (typed)
+                stream.ReadInt();
+
+            int len = stream.ReadInt();
+
+            var vals = new T[len];
+
+            for (int i = 0; i < len; i++)
+                vals[i] = ctx.Deserialize<T>();
+
+            return vals;
+        }
+
+        /**
+         * <summary>Read DateTime array.</summary>
+         * <param name="stream">Stream.</param>
+         * <param name="local">Local flag.</param>
+         * <returns>Array.</returns>
+         */
+        public static DateTime?[] ReadDateArray(IPortableStream stream, bool local)
+        {
+            int len = stream.ReadInt();
+
+            DateTime?[] vals = new DateTime?[len];
+
+            for (int i = 0; i < len; i++)
+                vals[i] = stream.ReadByte() == HdrNull ? null : ReadDate(stream, local);
+
+            return vals;
+        }
+
+        /**
+         * <summary>Write collection.</summary>
+         * <param name="val">Value.</param>
+         * <param name="ctx">Write context.</param>
+         */
+        public static void WriteCollection(ICollection val, PortableWriterImpl ctx)
+        {
+            byte colType = val.GetType() == typeof(ArrayList) ? CollectionArrayList : CollectionCustom;
+
+            WriteTypedCollection(val, ctx, colType);
+        }
+
+        /**
+         * <summary>Write non-null collection with known type.</summary>
+         * <param name="val">Value.</param>
+         * <param name="ctx">Write context.</param>
+         * <param name="colType">Collection type.</param>
+         */
+        public static void WriteTypedCollection(ICollection val, PortableWriterImpl ctx, byte colType)
+        {
+            ctx.Stream.WriteInt(val.Count);
+
+            ctx.Stream.WriteByte(colType);
+
+            foreach (object elem in val)
+                ctx.Write(elem);
+        }
+
+        /**
+         * <summary>Read collection.</summary>
+         * <param name="ctx">Context.</param>
+         * <param name="factory">Factory delegate.</param>
+         * <param name="adder">Adder delegate.</param>
+         * <returns>Collection.</returns>
+         */
+        public static ICollection ReadCollection(PortableReaderImpl ctx,
+            PortableCollectionFactory factory, PortableCollectionAdder adder)
+        {
+            if (factory == null)
+                factory = PortableSystemHandlers.CreateArrayList;
+
+            if (adder == null)
+                adder = PortableSystemHandlers.AddToArrayList;
+
+            IPortableStream stream = ctx.Stream;
+
+            int len = stream.ReadInt();
+
+            ctx.Stream.Seek(1, SeekOrigin.Current);
+
+            ICollection res = factory.Invoke(len);
+
+            for (int i = 0; i < len; i++)
+                adder.Invoke(res, ctx.Deserialize<object>());
+
+            return res;
+        }
+
+        /**
+         * <summary>Write generic collection.</summary>
+         * <param name="val">Value.</param>
+         * <param name="ctx">Write context.</param>
+         */
+        public static void WriteGenericCollection<T>(ICollection<T> val, PortableWriterImpl ctx)
+        {
+            Type type = val.GetType().GetGenericTypeDefinition();
+
+            byte colType;
+
+            if (type == typeof(List<>))
+                colType = CollectionArrayList;
+            else if (type == typeof(LinkedList<>))
+                colType = CollectionLinkedList;
+            else if (type == typeof(HashSet<>))
+                colType = CollectionHashSet;
+            else if (type == typeof(SortedSet<>))
+                colType = CollectionSortedSet;
+            else
+                colType = CollectionCustom;
+
+            WriteTypedGenericCollection(val, ctx, colType);
+        }
+
+        /**
+         * <summary>Write generic non-null collection with known type.</summary>
+         * <param name="val">Value.</param>
+         * <param name="ctx">Write context.</param>
+         * <param name="colType">Collection type.</param>
+         */
+        public static void WriteTypedGenericCollection<T>(ICollection<T> val, PortableWriterImpl ctx,
+            byte colType)
+        {
+            ctx.Stream.WriteInt(val.Count);
+
+            ctx.Stream.WriteByte(colType);
+
+            foreach (T elem in val)
+                ctx.Write(elem);
+        }
+
+        /**
+         * <summary>Read generic collection.</summary>
+         * <param name="ctx">Context.</param>
+         * <param name="factory">Factory delegate.</param>
+         * <returns>Collection.</returns>
+         */
+        public static ICollection<T> ReadGenericCollection<T>(PortableReaderImpl ctx,
+            PortableGenericCollectionFactory<T> factory)
+        {
+            int len = ctx.Stream.ReadInt();
+
+            if (len >= 0)
+            {
+                byte colType = ctx.Stream.ReadByte();
+
+                if (factory == null)
+                {
+                    // Need to detect factory automatically.
+                    if (colType == CollectionLinkedList)
+                        factory = PortableSystemHandlers.CreateLinkedList<T>;
+                    else if (colType == CollectionHashSet)
+                        factory = PortableSystemHandlers.CreateHashSet<T>;
+                    else if (colType == CollectionSortedSet)
+                        factory = PortableSystemHandlers.CreateSortedSet<T>;
+                    else
+                        factory = PortableSystemHandlers.CreateList<T>;
+                }
+
+                ICollection<T> res = factory.Invoke(len);
+
+                for (int i = 0; i < len; i++)
+                    res.Add(ctx.Deserialize<T>());
+
+                return res;
+            }
+            return null;
+        }
+
+        /**
+         * <summary>Write dictionary.</summary>
+         * <param name="val">Value.</param>
+         * <param name="ctx">Write context.</param>
+         */
+        public static void WriteDictionary(IDictionary val, PortableWriterImpl ctx)
+        {
+            byte dictType = val.GetType() == typeof(Hashtable) ? MapHashMap : MapCustom;
+
+            WriteTypedDictionary(val, ctx, dictType);
+        }
+
+        /**
+         * <summary>Write non-null dictionary with known type.</summary>
+         * <param name="val">Value.</param>
+         * <param name="ctx">Write context.</param>
+         * <param name="dictType">Dictionary type.</param>
+         */
+        public static void WriteTypedDictionary(IDictionary val, PortableWriterImpl ctx, byte dictType)
+        {
+            ctx.Stream.WriteInt(val.Count);
+
+            ctx.Stream.WriteByte(dictType);
+
+            foreach (DictionaryEntry entry in val)
+            {
+                ctx.Write(entry.Key);
+                ctx.Write(entry.Value);
+            }
+        }
+
+        /**
+         * <summary>Read dictionary.</summary>
+         * <param name="ctx">Context.</param>
+         * <param name="factory">Factory delegate.</param>
+         * <returns>Dictionary.</returns>
+         */
+        public static IDictionary ReadDictionary(PortableReaderImpl ctx,
+            PortableDictionaryFactory factory)
+        {
+            if (factory == null)
+                factory = PortableSystemHandlers.CreateHashtable;
+
+            IPortableStream stream = ctx.Stream;
+
+            int len = stream.ReadInt();
+
+            ctx.Stream.Seek(1, SeekOrigin.Current);
+
+            IDictionary res = factory.Invoke(len);
+
+            for (int i = 0; i < len; i++)
+            {
+                object key = ctx.Deserialize<object>();
+                object val = ctx.Deserialize<object>();
+
+                res[key] = val;
+            }
+
+            return res;
+        }
+
+        /**
+         * <summary>Write generic dictionary.</summary>
+         * <param name="val">Value.</param>
+         * <param name="ctx">Write context.</param>
+         */
+        public static void WriteGenericDictionary<TK, TV>(IDictionary<TK, TV> val, PortableWriterImpl ctx)
+        {
+            Type type = val.GetType().GetGenericTypeDefinition();
+
+            byte dictType;
+
+            if (type == typeof(Dictionary<,>))
+                dictType = MapHashMap;
+            else if (type == typeof(SortedDictionary<,>))
+                dictType = MapSortedMap;
+            else if (type == typeof(ConcurrentDictionary<,>))
+                dictType = MapConcurrentHashMap;
+            else
+                dictType = MapCustom;
+
+            WriteTypedGenericDictionary(val, ctx, dictType);
+        }
+
+        /**
+         * <summary>Write generic non-null dictionary with known type.</summary>
+         * <param name="val">Value.</param>
+         * <param name="ctx">Write context.</param>
+         * <param name="dictType">Dictionary type.</param>
+         */
+        public static void WriteTypedGenericDictionary<TK, TV>(IDictionary<TK, TV> val,
+            PortableWriterImpl ctx, byte dictType)
+        {
+            ctx.Stream.WriteInt(val.Count);
+
+            ctx.Stream.WriteByte(dictType);
+
+            foreach (KeyValuePair<TK, TV> entry in val)
+            {
+                ctx.Write(entry.Key);
+                ctx.Write(entry.Value);
+            }
+        }
+
+        /**
+         * <summary>Read generic dictionary.</summary>
+         * <param name="ctx">Context.</param>
+         * <param name="factory">Factory delegate.</param>
+         * <returns>Collection.</returns>
+         */
+        public static IDictionary<TK, TV> ReadGenericDictionary<TK, TV>(PortableReaderImpl ctx,
+            PortableGenericDictionaryFactory<TK, TV> factory)
+        {
+            int len = ctx.Stream.ReadInt();
+
+            if (len >= 0)
+            {
+                byte colType = ctx.Stream.ReadByte();
+
+                if (factory == null)
+                {
+                    if (colType == MapSortedMap)
+                        factory = PortableSystemHandlers.CreateSortedDictionary<TK, TV>;
+                    else if (colType == MapConcurrentHashMap)
+                        factory = PortableSystemHandlers.CreateConcurrentDictionary<TK, TV>;
+                    else
+                        factory = PortableSystemHandlers.CreateDictionary<TK, TV>;
+                }
+
+                IDictionary<TK, TV> res = factory.Invoke(len);
+
+                for (int i = 0; i < len; i++)
+                {
+                    TK key = ctx.Deserialize<TK>();
+                    TV val = ctx.Deserialize<TV>();
+
+                    res[key] = val;
+                }
+
+                return res;
+            }
+            return null;
+        }
+
+        /**
+         * <summary>Write map entry.</summary>
+         * <param name="ctx">Write context.</param>
+         * <param name="val">Value.</param>
+         */
+        public static void WriteMapEntry(PortableWriterImpl ctx, DictionaryEntry val)
+        {
+            ctx.Write(val.Key);
+            ctx.Write(val.Value);
+        }
+
+        /**
+         * <summary>Read map entry.</summary>
+         * <param name="ctx">Context.</param>
+         * <returns>Map entry.</returns>
+         */
+        public static DictionaryEntry ReadMapEntry(PortableReaderImpl ctx)
+        {
+            object key = ctx.Deserialize<object>();
+            object val = ctx.Deserialize<object>();
+
+            return new DictionaryEntry(key, val);
+        }
+
+        /**
+         * <summary>Write portable object.</summary>
+         * <param name="stream">Stream.</param>
+         * <param name="val">Value.</param>
+         */
+        public static void WritePortable(IPortableStream stream, PortableUserObject val)
+        {
+            WriteByteArray(val.Data, stream);
+
+            stream.WriteInt(val.Offset);
+        }
+
+        /// <summary>
+        /// Write enum.
+        /// </summary>
+        /// <param name="stream">Stream.</param>
+        /// <param name="val">Value.</param>
+        public static void WriteEnum(IPortableStream stream, Enum val)
+        {
+            if (Enum.GetUnderlyingType(val.GetType()) == TypInt)
+            {
+                stream.WriteInt(ObjTypeId);
+                stream.WriteInt(Convert.ToInt32(val));
+            }
+            else
+                throw new PortableException("Only Int32 underlying type is supported for enums: " +
+                    val.GetType().Name);
+        }
+
+        /// <summary>
+        /// Read enum.
+        /// </summary>
+        /// <param name="stream">Stream.</param>
+        /// <returns>Enumeration.</returns>
+        public static T ReadEnum<T>(IPortableStream stream)
+        {
+            if (!typeof(T).IsEnum || Enum.GetUnderlyingType(typeof(T)) == TypInt)
+            {
+                stream.ReadInt();
+
+                return TypeCaster<T>.Cast(stream.ReadInt());
+            }
+
+            throw new PortableException("Only Int32 underlying type is supported for enums: " +
+                                        typeof (T).Name);
+        }
+
+        /**
+         * <summary>Gets type key.</summary>
+         * <param name="userType">User type flag.</param>
+         * <param name="typeId">Type ID.</param>
+         * <returns>Type key.</returns>
+         */
+        public static long TypeKey(bool userType, int typeId)
+        {
+            long res = typeId;
+
+            if (userType)
+                res |= (long)1 << 32;
+
+            return res;
+        }
+
+        /**
+         * <summary>Get string hash code.</summary>
+         * <param name="val">Value.</param>
+         * <returns>Hash code.</returns>
+         */
+        public static int StringHashCode(string val)
+        {
+            if (val == null)
+                return 0;
+            int hash = 0;
+
+            for (int i = 0; i < val.Length; i++)
+            {
+                char c = val[i];
+
+                if ('A' <= c && c <= 'Z')
+                    c = (char)(c | 0x20);
+
+                hash = 31 * hash + c;
+            }
+
+            return hash;
+        }
+
+        public static string CleanFieldName(string fieldName)
+        {
+            if (fieldName.StartsWith("<") && fieldName.EndsWith(">k__BackingField"))
+                return fieldName.Substring(1, fieldName.IndexOf(">", StringComparison.Ordinal) - 1);
+            
+            return fieldName;
+        }
+
+        /**
+         * <summary>Check whether this is predefined type.</summary>
+         * <param name="hdr">Header.</param>
+         * <returns>True is this is one of predefined types with special semantics.</returns>
+         */
+        public static bool IsPredefinedType(byte hdr)
+        {
+            switch (hdr)
+            {
+                case TypeByte:
+                case TypeShort:
+                case TypeInt:
+                case TypeLong:
+                case TypeFloat:
+                case TypeDouble:
+                case TypeChar:
+                case TypeBool:
+                case TypeDecimal:
+                case TypeString:
+                case TypeGuid:
+                case TypeDate:
+                case TypeEnum:
+                case TypeArrayByte:
+                case TypeArrayShort:
+                case TypeArrayInt:
+                case TypeArrayLong:
+                case TypeArrayFloat:
+                case TypeArrayDouble:
+                case TypeArrayChar:
+                case TypeArrayBool:
+                case TypeArrayDecimal:
+                case TypeArrayString:
+                case TypeArrayGuid:
+                case TypeArrayDate:
+                case TypeArrayEnum:
+                case TypeArray:
+                case TypeCollection:
+                case TypeDictionary:
+                case TypeMapEntry:
+                case TypePortable:
+                    return true;
+                default:
+                    return false;
+            }
+        }
+
+        /**
+         * <summary>Convert type name.</summary>
+         * <param name="typeName">Type name.</param>
+         * <param name="converter">Converter.</param>
+         * <returns>Converted name.</returns>
+         */
+        public static string ConvertTypeName(string typeName, IPortableNameMapper converter)
+        {
+            var typeName0 = typeName;
+
+            try
+            {
+                if (converter != null)
+                    typeName = converter.TypeName(typeName);
+            }
+            catch (Exception e)
+            {
+                throw new PortableException("Failed to convert type name due to converter exception " +
+                    "[typeName=" + typeName + ", converter=" + converter + ']', e);
+            }
+
+            if (typeName == null)
+                throw new PortableException("Name converter returned null name for type [typeName=" +
+                    typeName0 + ", converter=" + converter + "]");
+
+            return typeName;
+        }
+
+        /**
+         * <summary>Convert field name.</summary>
+         * <param name="fieldName">Field name.</param>
+         * <param name="converter">Converter.</param>
+         * <returns>Converted name.</returns>
+         */
+        public static string ConvertFieldName(string fieldName, IPortableNameMapper converter)
+        {
+            var fieldName0 = fieldName;
+
+            try
+            {
+                if (converter != null)
+                    fieldName = converter.FieldName(fieldName);
+            }
+            catch (Exception e)
+            {
+                throw new PortableException("Failed to convert field name due to converter exception " +
+                    "[fieldName=" + fieldName + ", converter=" + converter + ']', e);
+            }
+
+            if (fieldName == null)
+                throw new PortableException("Name converter returned null name for field [fieldName=" +
+                    fieldName0 + ", converter=" + converter + "]");
+
+            return fieldName;
+        }
+
+        /**
+         * <summary>Extract simple type name.</summary>
+         * <param name="typeName">Type name.</param>
+         * <returns>Simple type name.</returns>
+         */
+        public static string SimpleTypeName(string typeName)
+        {
+            int idx = typeName.LastIndexOf('.');
+
+            return idx < 0 ? typeName : typeName.Substring(idx + 1);
+        }
+
+        /**
+         * <summary>Resolve type ID.</summary>
+         * <param name="typeName">Type name.</param>
+         * <param name="nameMapper">Name mapper.</param>
+         * <param name="idMapper">ID mapper.</param>
+         */
+        public static int TypeId(string typeName, IPortableNameMapper nameMapper,
+            IPortableIdMapper idMapper)
+        {
+            Debug.Assert(typeName != null);
+
+            typeName = ConvertTypeName(typeName, nameMapper);
+
+            int id = 0;
+
+            if (idMapper != null)
+            {
+                try
+                {
+                    id = idMapper.TypeId(typeName);
+                }
+                catch (Exception e)
+                {
+                    throw new PortableException("Failed to resolve type ID due to ID mapper exception " +
+                        "[typeName=" + typeName + ", idMapper=" + idMapper + ']', e);
+                }
+            }
+
+            if (id == 0)
+                id = StringHashCode(typeName);
+
+            return id;
+        }
+
+        /**
+         * <summary>Resolve field ID.</summary>
+         * <param name="typeId">Type ID.</param>
+         * <param name="fieldName">Field name.</param>
+         * <param name="nameMapper">Name mapper.</param>
+         * <param name="idMapper">ID mapper.</param>
+         */
+        public static int FieldId(int typeId, string fieldName, IPortableNameMapper nameMapper,
+            IPortableIdMapper idMapper)
+        {
+            Debug.Assert(typeId != 0);
+            Debug.Assert(fieldName != null);
+
+            fieldName = ConvertFieldName(fieldName, nameMapper);
+
+            int id = 0;
+
+            if (idMapper != null)
+            {
+                try
+                {
+                    id = idMapper.FieldId(typeId, fieldName);
+                }
+                catch (Exception e)
+                {
+                    throw new PortableException("Failed to resolve field ID due to ID mapper exception " +
+                        "[typeId=" + typeId + ", fieldName=" + fieldName + ", idMapper=" + idMapper + ']', e);
+                }
+            }
+
+            if (id == 0)
+                id = StringHashCode(fieldName);
+
+            return id;
+        }
+
+        /**
+         * <summary>Get fields map for the given object.</summary>
+         * <param name="stream">Stream.</param>
+         * <param name="typeId">Type ID.</param>
+         * <param name="rawDataOffset">Raw data offset.</param>
+         * <returns>Dictionary with field ID as key and field position as value.</returns>
+         */
+        public static IDictionary<int, int> ObjectFields(IPortableStream stream, int typeId, int rawDataOffset)
+        {
+            int endPos = stream.Position + rawDataOffset - 18;
+
+            // First loop detects amount of fields in the object.
+            int retPos = stream.Position;
+            int cnt = 0;
+
+            while (stream.Position < endPos)
+            {
+                cnt++;
+
+                stream.Seek(4, SeekOrigin.Current);
+                int len = stream.ReadInt();
+
+                stream.Seek(stream.Position + len, SeekOrigin.Begin);
+            }
+
+            if (cnt == 0)
+                return EmptyFields;
+
+            stream.Seek(retPos, SeekOrigin.Begin);
+
+            IDictionary<int, int> fields = new Dictionary<int, int>(cnt);
+
+            // Second loop populates fields.
+            while (stream.Position < endPos)
+            {
+                int id = stream.ReadInt();
+                int len = stream.ReadInt();
+
+                if (fields.ContainsKey(id))
+                    throw new PortableException("Object contains duplicate field IDs [typeId=" +
+                        typeId + ", fieldId=" + id + ']');
+
+                fields[id] = stream.Position; // Add field ID and length.
+
+                stream.Seek(stream.Position + len, SeekOrigin.Begin);
+            }
+
+            return fields;
+        }
+
+        /// <summary>
+        /// Compare contents of two byte array chunks.
+        /// </summary>
+        /// <param name="arr1">Array 1.</param>
+        /// <param name="offset1">Offset 1.</param>
+        /// <param name="len1">Length 1.</param>
+        /// <param name="arr2">Array 2.</param>
+        /// <param name="offset2">Offset 2.</param>
+        /// <param name="len2">Length 2.</param>
+        /// <returns>True if array chunks are equal.</returns>
+        public static bool CompareArrays(byte[] arr1, int offset1, int len1, byte[] arr2, int offset2, int len2)
+        {
+            if (len1 == len2)
+            {
+                for (int i = 0; i < len1; i++)
+                {
+                    if (arr1[offset1 + i] != arr2[offset2 + i])
+                        return false;
+                }
+
+                return true;
+            }
+            return false;
+        }
+
+        /// <summary>
+        /// Write object which is not necessary portable.
+        /// </summary>
+        /// <param name="writer">Writer.</param>
+        /// <param name="obj">Object.</param>
+        public static void WritePortableOrSerializable<T>(PortableWriterImpl writer, T obj)
+        {
+            if (writer.IsPortable(obj))
+            {
+                writer.WriteBoolean(true);
+
+                writer.WriteObject(obj);
+            }
+            else
+            {
+                writer.WriteBoolean(false);
+
+                WriteSerializable(writer, obj);
+            }
+        }
+
+        /// <summary>
+        /// Writes a serializable object.
+        /// </summary>
+        /// <param name="writer">Writer.</param>
+        /// <param name="obj">Object.</param>
+        public static void WriteSerializable<T>(PortableWriterImpl writer, T obj)
+        {
+            new BinaryFormatter().Serialize(new PortableStreamAdapter(writer.Stream), obj);
+        }
+
+        /// <summary>
+        /// Read object which is not necessary portable.
+        /// </summary>
+        /// <param name="reader">Reader.</param>
+        /// <returns>Object.</returns>
+        public static T ReadPortableOrSerializable<T>(PortableReaderImpl reader)
+        {
+            return reader.ReadBoolean()
+                ? reader.ReadObject<T>()
+                : ReadSerializable<T>(reader);
+        }
+
+        /// <summary>
+        /// Reads a serializable object.
+        /// </summary>
+        /// <param name="reader">Reader.</param>
+        /// <returns>Object.</returns>
+        public static T ReadSerializable<T>(PortableReaderImpl reader)
+        {
+            return (T) new BinaryFormatter().Deserialize(new PortableStreamAdapter(reader.Stream), null);
+        }
+
+        /// <summary>
+        /// Writes wrapped invocation result.
+        /// </summary>
+        /// <param name="writer">Writer.</param>
+        /// <param name="success">Success flag.</param>
+        /// <param name="res">Result.</param>
+        public static void WriteWrappedInvocationResult(PortableWriterImpl writer, bool success, object res)
+        {
+            var pos = writer.Stream.Position;
+
+            try
+            {
+                if (success)
+                    writer.WriteBoolean(true);
+                else
+                {
+                    writer.WriteBoolean(false); // Call failed.
+                    writer.WriteBoolean(true); // Exception serialized sucessfully.
+                }
+
+                writer.Write(new PortableResultWrapper(res));
+            }
+            catch (Exception marshErr)
+            {
+                // Failed to serialize result, fallback to plain string.
+                writer.Stream.Seek(pos, SeekOrigin.Begin);
+
+                writer.WriteBoolean(false); // Call failed.
+                writer.WriteBoolean(false); // Cannot serialize result or exception.
+
+                if (success)
+                {
+                    writer.WriteString("Call completed successfully, but result serialization failed [resultType=" +
+                        res.GetType().Name + ", serializationErrMsg=" + marshErr.Message + ']');
+                }
+                else
+                {
+                    writer.WriteString("Call completed with error, but error serialization failed [errType=" +
+                        res.GetType().Name + ", serializationErrMsg=" + marshErr.Message + ']');
+                }
+            }
+        }
+        /// <summary>
+        /// Writes invocation result.
+        /// </summary>
+        /// <param name="writer">Writer.</param>
+        /// <param name="success">Success flag.</param>
+        /// <param name="res">Result.</param>
+        public static void WriteInvocationResult(PortableWriterImpl writer, bool success, object res)
+        {
+            var pos = writer.Stream.Position;
+
+            try
+            {
+                if (success)
+                    writer.WriteBoolean(true);
+                else
+                {
+                    writer.WriteBoolean(false); // Call failed.
+                    writer.WriteBoolean(true); // Exception serialized sucessfully.
+                }
+
+                writer.Write(res);
+            }
+            catch (Exception marshErr)
+            {
+                // Failed to serialize result, fallback to plain string.
+                writer.Stream.Seek(pos, SeekOrigin.Begin);
+
+                writer.WriteBoolean(false); // Call failed.
+                writer.WriteBoolean(false); // Cannot serialize result or exception.
+
+                if (success)
+                {
+                    writer.WriteString("Call completed successfully, but result serialization failed [resultType=" +
+                        res.GetType().Name + ", serializationErrMsg=" + marshErr.Message + ']');
+                }
+                else
+                {
+                    writer.WriteString("Call completed with error, but error serialization failed [errType=" +
+                        res.GetType().Name + ", serializationErrMsg=" + marshErr.Message + ']');
+                }
+            }
+        }
+
+        /// <summary>
+        /// Reads wrapped invocation result.
+        /// </summary>
+        /// <param name="reader">Reader.</param>
+        /// <param name="err">Error.</param>
+        /// <returns>Result.</returns>
+        public static object ReadWrappedInvocationResult(PortableReaderImpl reader, out object err)
+        {
+            err = null;
+
+            if (reader.ReadBoolean())
+                return reader.ReadObject<PortableResultWrapper>().Result;
+
+            if (reader.ReadBoolean())
+                err = (Exception) reader.ReadObject<PortableResultWrapper>().Result;
+            else
+                err = ExceptionUtils.GetException(reader.ReadString(), reader.ReadString());
+
+            return null;
+        }
+
+        /// <summary>
+        /// Reads invocation result.
+        /// </summary>
+        /// <param name="reader">Reader.</param>
+        /// <param name="err">Error.</param>
+        /// <returns>Result.</returns>
+        public static object ReadInvocationResult(PortableReaderImpl reader, out object err)
+        {
+            err = null;
+
+            if (reader.ReadBoolean())
+                return reader.ReadObject<object>();
+
+            if (reader.ReadBoolean())
+                err = reader.ReadObject<object>();
+            else
+                err = ExceptionUtils.GetException(reader.ReadString(), reader.ReadString());
+
+            return null;
+        }
+
+        /**
+         * <summary>Convert date to Java ticks.</summary>
+         * <param name="date">Date</param>
+         * <param name="high">High part (milliseconds).</param>
+         * <param name="low">Low part (nanoseconds)</param>
+         */
+        private static void ToJavaDate(DateTime date, out long high, out int low)
+        {
+            long diff = date.ToUniversalTime().Ticks - JavaDateTicks;
+
+            high = diff / TimeSpan.TicksPerMillisecond;
+
+            low = (int)(diff % TimeSpan.TicksPerMillisecond) * 100; 
+        }
+
+        /**
+         * <summary>Convert Java ticks to date.</summary>
+         * <param name="high">High part (milliseconds).</param>
+         * <param name="low">Low part (nanoseconds).</param>
+         * <param name="local">Whether the time should be treaten as local.</param>
+         * <returns>Date.</returns>
+         */
+        private static DateTime ToDotNetDate(long high, int low, bool local)
+        {
+            DateTime res = 
+                new DateTime(JavaDateTicks + high * TimeSpan.TicksPerMillisecond + low / 100, DateTimeKind.Utc);
+
+            return local ? res.ToLocalTime() : res;
+        }
+    }
+}


[24/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableBuilderImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableBuilderImpl.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableBuilderImpl.cs
new file mode 100644
index 0000000..037ac85
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableBuilderImpl.cs
@@ -0,0 +1,923 @@
+/*
+ * 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.Portable
+{
+    using System;
+    using System.Collections.Generic;
+    using System.IO;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Impl.Portable.Metadata;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Portable builder implementation.
+    /// </summary>
+    internal class PortableBuilderImpl : IPortableBuilder
+    {
+        /** Type IDs for metadata. */
+        private static readonly IDictionary<Type, int> TypeIds;
+
+        /** Cached dictionary with no values. */
+        private static readonly IDictionary<int, object> EmptyVals = new Dictionary<int, object>();
+
+        /** Offset: length. */
+        private const int OffsetLen = 10;
+
+        /** Portables. */
+        private readonly PortablesImpl _portables;
+
+        /** */
+        private readonly PortableBuilderImpl _parent;
+
+        /** Initial portable object. */
+        private readonly PortableUserObject _obj;
+
+        /** Type descriptor. */
+        private readonly IPortableTypeDescriptor _desc;
+
+        /** Values. */
+        private IDictionary<string, PortableBuilderField> _vals;
+
+        /** Contextual fields. */
+        private IDictionary<int, object> _cache;
+
+        /** Hash code. */
+        private int _hashCode;
+        
+        /** Current context. */
+        private Context _ctx;
+        
+        /// <summary>
+        /// Static initializer.
+        /// </summary>
+        static PortableBuilderImpl()
+        {
+            TypeIds = new Dictionary<Type, int>();
+
+            // 1. Primitives.
+            TypeIds[typeof(byte)] = PortableUtils.TypeByte;
+            TypeIds[typeof(bool)] = PortableUtils.TypeBool;
+            TypeIds[typeof(short)] = PortableUtils.TypeShort;
+            TypeIds[typeof(char)] = PortableUtils.TypeChar;
+            TypeIds[typeof(int)] = PortableUtils.TypeInt;
+            TypeIds[typeof(long)] = PortableUtils.TypeLong;
+            TypeIds[typeof(float)] = PortableUtils.TypeFloat;
+            TypeIds[typeof(double)] = PortableUtils.TypeDouble;
+            TypeIds[typeof(decimal)] = PortableUtils.TypeDecimal;
+
+            TypeIds[typeof(byte[])] = PortableUtils.TypeArrayByte;
+            TypeIds[typeof(bool[])] = PortableUtils.TypeArrayBool;
+            TypeIds[typeof(short[])] = PortableUtils.TypeArrayShort;
+            TypeIds[typeof(char[])] = PortableUtils.TypeArrayChar;
+            TypeIds[typeof(int[])] = PortableUtils.TypeArrayInt;
+            TypeIds[typeof(long[])] = PortableUtils.TypeArrayLong;
+            TypeIds[typeof(float[])] = PortableUtils.TypeArrayFloat;
+            TypeIds[typeof(double[])] = PortableUtils.TypeArrayDouble;
+            TypeIds[typeof(decimal[])] = PortableUtils.TypeArrayDecimal;
+
+            // 2. String.
+            TypeIds[typeof(string)] = PortableUtils.TypeString;
+            TypeIds[typeof(string[])] = PortableUtils.TypeArrayString;
+
+            // 3. Guid.
+            TypeIds[typeof(Guid)] = PortableUtils.TypeGuid;
+            TypeIds[typeof(Guid?)] = PortableUtils.TypeGuid;
+            TypeIds[typeof(Guid[])] = PortableUtils.TypeArrayGuid;
+            TypeIds[typeof(Guid?[])] = PortableUtils.TypeArrayGuid;
+
+            // 4. Date.
+            TypeIds[typeof(DateTime)] = PortableUtils.TypeDate;
+            TypeIds[typeof(DateTime?)] = PortableUtils.TypeDate;
+            TypeIds[typeof(DateTime[])] = PortableUtils.TypeArrayDate;
+            TypeIds[typeof(DateTime?[])] = PortableUtils.TypeArrayDate;
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="portables">Portables.</param>
+        /// <param name="obj">Initial portable object.</param>
+        /// <param name="desc">Type descriptor.</param>
+        public PortableBuilderImpl(PortablesImpl portables, PortableUserObject obj,
+            IPortableTypeDescriptor desc) : this(portables, null, obj, desc) 
+        { 
+            // No-op.
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="portables">Portables.</param>
+        /// <param name="parent">Parent builder.</param>
+        /// <param name="obj">Initial portable object.</param>
+        /// <param name="desc">Type descriptor.</param>
+        public PortableBuilderImpl(PortablesImpl portables, PortableBuilderImpl parent, 
+            PortableUserObject obj, IPortableTypeDescriptor desc)
+        {
+            _portables = portables;
+            _parent = parent ?? this;
+            _obj = obj;
+            _desc = desc;
+
+            _hashCode = obj.GetHashCode();
+        }
+
+        /** <inheritDoc /> */
+        public IPortableBuilder HashCode(int hashCode)
+        {
+            _hashCode = hashCode;
+
+            return this;
+        }
+
+        /** <inheritDoc /> */
+        public T GetField<T>(string name)
+        {
+            PortableBuilderField field;
+
+            if (_vals != null && _vals.TryGetValue(name, out field))
+                return field != PortableBuilderField.RmvMarker ? (T)field.Value : default(T);
+            T val = _obj.Field<T>(name, this);
+
+            if (_vals == null)
+                _vals = new Dictionary<string, PortableBuilderField>(2);
+
+            _vals[name] = new PortableBuilderField(typeof(T), val);
+
+            return val;
+        }
+
+        /** <inheritDoc /> */
+        public IPortableBuilder SetField<T>(string name, T val)
+        {
+            return SetField0(name, new PortableBuilderField(typeof(T), val));
+        }
+
+        /** <inheritDoc /> */
+        public IPortableBuilder RemoveField(string name)
+        {
+            return SetField0(name, PortableBuilderField.RmvMarker);
+        }
+
+        /** <inheritDoc /> */
+        public IPortableObject Build()
+        {
+            PortableHeapStream inStream = new PortableHeapStream(_obj.Data);
+
+            inStream.Seek(_obj.Offset, SeekOrigin.Begin);
+
+            // Assume that resulting length will be no less than header + [fields_cnt] * 12;
+            int len = PortableUtils.FullHdrLen + (_vals == null ? 0 : _vals.Count * 12);
+
+            PortableHeapStream outStream = new PortableHeapStream(len);
+
+            PortableWriterImpl writer = _portables.Marshaller.StartMarshal(outStream);
+
+            writer.Builder(this);
+
+            // All related builders will work in this context with this writer.
+            _parent._ctx = new Context(writer);
+            
+            try
+            {
+                // Write.
+                writer.Write(this, null);
+                
+                // Process metadata.
+                _portables.Marshaller.FinishMarshal(writer);
+
+                // Create portable object once metadata is processed.
+                return new PortableUserObject(_portables.Marshaller, outStream.InternalArray, 0,
+                    _desc.TypeId, _hashCode);
+            }
+            finally
+            {
+                // Cleanup.
+                _parent._ctx.Closed = true;
+            }
+        }
+
+        /// <summary>
+        /// Create child builder.
+        /// </summary>
+        /// <param name="obj">Portable object.</param>
+        /// <returns>Child builder.</returns>
+        public PortableBuilderImpl Child(PortableUserObject obj)
+        {
+            return _portables.ChildBuilder(_parent, obj);
+        }
+        
+        /// <summary>
+        /// Get cache field.
+        /// </summary>
+        /// <param name="pos">Position.</param>
+        /// <param name="val">Value.</param>
+        /// <returns><c>true</c> if value is found in cache.</returns>
+        public bool CachedField<T>(int pos, out T val)
+        {
+            if (_parent._cache != null)
+            {
+                object res;
+
+                if (_parent._cache.TryGetValue(pos, out res))
+                {
+                    val = res != null ? (T)res : default(T);
+
+                    return true;
+                }
+            }
+
+            val = default(T);
+
+            return false;
+        }
+
+        /// <summary>
+        /// Add field to cache test.
+        /// </summary>
+        /// <param name="pos">Position.</param>
+        /// <param name="val">Value.</param>
+        public void CacheField(int pos, object val)
+        {
+            if (_parent._cache == null)
+                _parent._cache = new Dictionary<int, object>(2);
+
+            _parent._cache[pos] = val;
+        }
+
+        /// <summary>
+        /// Internal set field routine.
+        /// </summary>
+        /// <param name="fieldName">Name.</param>
+        /// <param name="val">Value.</param>
+        /// <returns>This builder.</returns>
+        private IPortableBuilder SetField0(string fieldName, PortableBuilderField val)
+        {
+            if (_vals == null)
+                _vals = new Dictionary<string, PortableBuilderField>();
+
+            _vals[fieldName] = val;
+
+            return this;
+        }
+
+        /// <summary>
+        /// Mutate portable object.
+        /// </summary>
+        /// <param name="inStream">Input stream with initial object.</param>
+        /// <param name="outStream">Output stream.</param>
+        /// <param name="desc">Portable type descriptor.</param>
+        /// <param name="hashCode">Hash code.</param>
+        /// <param name="vals">Values.</param>
+        internal void Mutate(
+            PortableHeapStream inStream,
+            PortableHeapStream outStream,
+            IPortableTypeDescriptor desc,
+            int hashCode, 
+            IDictionary<string, PortableBuilderField> vals)
+        {
+            // Set correct builder to writer frame.
+            PortableBuilderImpl oldBuilder = _parent._ctx.Writer.Builder(_parent);
+
+            int streamPos = inStream.Position;
+            
+            try
+            {
+                // Prepare fields.
+                IPortableMetadataHandler metaHnd = _portables.Marshaller.MetadataHandler(desc);
+
+                IDictionary<int, object> vals0;
+
+                if (vals == null || vals.Count == 0)
+                    vals0 = EmptyVals;
+                else
+                {
+                    vals0 = new Dictionary<int, object>(vals.Count);
+
+                    foreach (KeyValuePair<string, PortableBuilderField> valEntry in vals)
+                    {
+                        int fieldId = PortableUtils.FieldId(desc.TypeId, valEntry.Key, desc.NameConverter, desc.Mapper);
+
+                        if (vals0.ContainsKey(fieldId))
+                            throw new IgniteException("Collision in field ID detected (change field name or " +
+                                "define custom ID mapper) [fieldName=" + valEntry.Key + ", fieldId=" + fieldId + ']');
+
+                        vals0[fieldId] = valEntry.Value.Value;
+
+                        // Write metadata if: 1) it is enabled for type; 2) type is not null (i.e. it is neither 
+                        // remove marker, nor a field read through "GetField" method.
+                        if (metaHnd != null && valEntry.Value.Type != null)
+                            metaHnd.OnFieldWrite(fieldId, valEntry.Key, TypeId(valEntry.Value.Type));
+                    }
+                }
+
+                // Actual processing.
+                Mutate0(_parent._ctx, inStream, outStream, true, hashCode, vals0);
+
+                // 3. Handle metadata.
+                if (metaHnd != null)
+                {
+                    IDictionary<string, int> meta = metaHnd.OnObjectWriteFinished();
+
+                    if (meta != null)
+                        _parent._ctx.Writer.SaveMetadata(desc.TypeId, desc.TypeName, desc.AffinityKeyFieldName, meta);
+                }
+            }
+            finally
+            {
+                // Restore builder frame.
+                _parent._ctx.Writer.Builder(oldBuilder);
+
+                inStream.Seek(streamPos, SeekOrigin.Begin);
+            }
+        }
+
+        /// <summary>
+        /// Internal mutation routine.
+        /// </summary>
+        /// <param name="inStream">Input stream.</param>
+        /// <param name="outStream">Output stream.</param>
+        /// <param name="ctx">Context.</param>
+        /// <param name="changeHash">WHether hash should be changed.</param>
+        /// <param name="hash">New hash.</param>
+        /// <param name="vals">Values to be replaced.</param>
+        /// <returns>Mutated object.</returns>
+        private void Mutate0(Context ctx, PortableHeapStream inStream, IPortableStream outStream,
+            bool changeHash, int hash, IDictionary<int, object> vals)
+        {
+            int inStartPos = inStream.Position;
+            int outStartPos = outStream.Position;
+
+            byte inHdr = inStream.ReadByte();
+
+            if (inHdr == PortableUtils.HdrNull)
+                outStream.WriteByte(PortableUtils.HdrNull);
+            else if (inHdr == PortableUtils.HdrHnd)
+            {
+                int inHnd = inStream.ReadInt();
+
+                int oldPos = inStartPos - inHnd;
+                int newPos;
+
+                if (ctx.OldToNew(oldPos, out newPos))
+                {
+                    // Handle is still valid.
+                    outStream.WriteByte(PortableUtils.HdrHnd);
+                    outStream.WriteInt(outStartPos - newPos);
+                }
+                else
+                {
+                    // Handle is invalid, write full object.
+                    int inRetPos = inStream.Position;
+
+                    inStream.Seek(oldPos, SeekOrigin.Begin);
+
+                    Mutate0(ctx, inStream, outStream, false, 0, EmptyVals);
+
+                    inStream.Seek(inRetPos, SeekOrigin.Begin);
+                }
+            }
+            else if (inHdr == PortableUtils.HdrFull)
+            {
+                byte inUsrFlag = inStream.ReadByte();
+                int inTypeId = inStream.ReadInt();
+                int inHash = inStream.ReadInt();
+                int inLen = inStream.ReadInt();
+                int inRawOff = inStream.ReadInt();
+
+                int hndPos;
+
+                if (ctx.AddOldToNew(inStartPos, outStartPos, out hndPos))
+                {
+                    // Object could be cached in parent builder.
+                    object cachedVal;
+
+                    if (_parent._cache != null && _parent._cache.TryGetValue(inStartPos, out cachedVal)) {
+                        ctx.Writer.Write(cachedVal, null);
+                    }
+                    else
+                    {
+                        // New object, write in full form.
+                        outStream.WriteByte(PortableUtils.HdrFull);
+                        outStream.WriteByte(inUsrFlag);
+                        outStream.WriteInt(inTypeId);
+                        outStream.WriteInt(changeHash ? hash : inHash);
+
+                        // Skip length and raw offset as they are not known at this point.
+                        outStream.Seek(8, SeekOrigin.Current);
+
+                        // Write regular fields.
+                        while (inStream.Position < inStartPos + inRawOff)
+                        {
+                            int inFieldId = inStream.ReadInt();
+                            int inFieldLen = inStream.ReadInt();
+                            int inFieldDataPos = inStream.Position;
+
+                            object fieldVal;
+
+                            bool fieldFound = vals.TryGetValue(inFieldId, out fieldVal);
+
+                            if (!fieldFound || fieldVal != PortableBuilderField.RmvMarkerObj)
+                            {
+                                outStream.WriteInt(inFieldId);
+
+                                int fieldLenPos = outStream.Position; // Here we will write length later.
+
+                                outStream.Seek(4, SeekOrigin.Current);
+
+                                if (fieldFound)
+                                {
+                                    // Replace field with new value.
+                                    if (fieldVal != PortableBuilderField.RmvMarkerObj)
+                                        ctx.Writer.Write(fieldVal, null);
+
+                                    vals.Remove(inFieldId);
+                                }
+                                else
+                                {
+                                    // If field was requested earlier, then we must write tracked value
+                                    if (_parent._cache != null && _parent._cache.TryGetValue(inFieldDataPos, out fieldVal))
+                                        ctx.Writer.Write(fieldVal, null);
+                                    else
+                                        // Filed is not tracked, re-write as is.
+                                        Mutate0(ctx, inStream, outStream, false, 0, EmptyVals);                                    
+                                }
+
+                                int fieldEndPos = outStream.Position;
+
+                                outStream.Seek(fieldLenPos, SeekOrigin.Begin);
+                                outStream.WriteInt(fieldEndPos - fieldLenPos - 4);
+                                outStream.Seek(fieldEndPos, SeekOrigin.Begin);
+                            }
+
+                            // Position intput stream pointer after the field.
+                            inStream.Seek(inFieldDataPos + inFieldLen, SeekOrigin.Begin);
+                        }
+
+                        // Write remaining new fields.
+                        foreach (KeyValuePair<int, object> valEntry in vals)
+                        {
+                            if (valEntry.Value != PortableBuilderField.RmvMarkerObj)
+                            {
+                                outStream.WriteInt(valEntry.Key);
+
+                                int fieldLenPos = outStream.Position; // Here we will write length later.
+
+                                outStream.Seek(4, SeekOrigin.Current);
+
+                                ctx.Writer.Write(valEntry.Value, null);
+
+                                int fieldEndPos = outStream.Position;
+
+                                outStream.Seek(fieldLenPos, SeekOrigin.Begin);
+                                outStream.WriteInt(fieldEndPos - fieldLenPos - 4);
+                                outStream.Seek(fieldEndPos, SeekOrigin.Begin);
+                            }
+                        }
+
+                        // Write raw data.
+                        int rawPos = outStream.Position;
+
+                        outStream.Write(inStream.InternalArray, inStartPos + inRawOff, inLen - inRawOff);
+
+                        // Write length and raw data offset.
+                        int outResPos = outStream.Position;
+
+                        outStream.Seek(outStartPos + OffsetLen, SeekOrigin.Begin);
+
+                        outStream.WriteInt(outResPos - outStartPos); // Length.
+                        outStream.WriteInt(rawPos - outStartPos); // Raw offset.
+
+                        outStream.Seek(outResPos, SeekOrigin.Begin);
+                    }
+                }
+                else
+                {
+                    // Object has already been written, write as handle.
+                    outStream.WriteByte(PortableUtils.HdrHnd);
+                    outStream.WriteInt(outStartPos - hndPos);
+                }
+
+                // Synchronize input stream position.
+                inStream.Seek(inStartPos + inLen, SeekOrigin.Begin);
+            }
+            else
+            {
+                // Try writing as well-known type with fixed size.
+                outStream.WriteByte(inHdr);
+
+                if (!WriteAsPredefined(inHdr, inStream, outStream, ctx))
+                    throw new IgniteException("Unexpected header [position=" + (inStream.Position - 1) +
+                        ", header=" + inHdr + ']');
+            }
+        }
+
+        /// <summary>
+        /// Process portable object inverting handles if needed.
+        /// </summary>
+        /// <param name="outStream">Output stream.</param>
+        /// <param name="port">Portable.</param>
+        internal void ProcessPortable(IPortableStream outStream, PortableUserObject port)
+        {
+            // Special case: writing portable object with correct inversions.
+            PortableHeapStream inStream = new PortableHeapStream(port.Data);
+
+            inStream.Seek(port.Offset, SeekOrigin.Begin);
+
+            // Use fresh context to ensure correct portable inversion.
+            Mutate0(new Context(), inStream, outStream, false, 0, EmptyVals);
+        }
+
+        /// <summary>
+        /// Process child builder.
+        /// </summary>
+        /// <param name="outStream">Output stream.</param>
+        /// <param name="builder">Builder.</param>
+        internal void ProcessBuilder(IPortableStream outStream, PortableBuilderImpl builder)
+        {
+            PortableHeapStream inStream = new PortableHeapStream(builder._obj.Data);
+
+            inStream.Seek(builder._obj.Offset, SeekOrigin.Begin);
+
+            // Builder parent context might be null only in one case: if we never met this group of
+            // builders before. In this case we set context to their parent and track it. Context
+            // cleanup will be performed at the very end of build process.
+            if (builder._parent._ctx == null || builder._parent._ctx.Closed)
+                builder._parent._ctx = new Context(_parent._ctx);
+
+            builder.Mutate(inStream, outStream as PortableHeapStream, builder._desc,
+                    builder._hashCode, builder._vals);
+        }
+
+        /// <summary>
+        /// Write object as a predefined type if possible.
+        /// </summary>
+        /// <param name="hdr">Header.</param>
+        /// <param name="inStream">Input stream.</param>
+        /// <param name="outStream">Output stream.</param>
+        /// <param name="ctx">Context.</param>
+        /// <returns><c>True</c> if was written.</returns>
+        private bool WriteAsPredefined(byte hdr, PortableHeapStream inStream, IPortableStream outStream,
+            Context ctx)
+        {
+            switch (hdr)
+            {
+                case PortableUtils.TypeByte:
+                    TransferBytes(inStream, outStream, 1);
+
+                    break;
+
+                case PortableUtils.TypeShort:
+                    TransferBytes(inStream, outStream, 2);
+
+                    break;
+
+                case PortableUtils.TypeInt:
+                    TransferBytes(inStream, outStream, 4);
+
+                    break;
+
+                case PortableUtils.TypeLong:
+                    TransferBytes(inStream, outStream, 8);
+
+                    break;
+
+                case PortableUtils.TypeFloat:
+                    TransferBytes(inStream, outStream, 4);
+
+                    break;
+
+                case PortableUtils.TypeDouble:
+                    TransferBytes(inStream, outStream, 8);
+
+                    break;
+
+                case PortableUtils.TypeChar:
+                    TransferBytes(inStream, outStream, 2);
+
+                    break;
+
+                case PortableUtils.TypeBool:
+                    TransferBytes(inStream, outStream, 1);
+
+                    break;
+
+                case PortableUtils.TypeDecimal:
+                    TransferBytes(inStream, outStream, 4); // Transfer scale
+
+                    int magLen = inStream.ReadInt(); // Transfer magnitude length.
+
+                    outStream.WriteInt(magLen);
+
+                    TransferBytes(inStream, outStream, magLen); // Transfer magnitude.
+
+                    break;
+
+                case PortableUtils.TypeString:
+                    PortableUtils.WriteString(PortableUtils.ReadString(inStream), outStream);
+
+                    break;
+
+                case PortableUtils.TypeGuid:
+                    TransferBytes(inStream, outStream, 16);
+
+                    break;
+
+                case PortableUtils.TypeDate:
+                    TransferBytes(inStream, outStream, 12);
+
+                    break;
+
+                case PortableUtils.TypeArrayByte:
+                    TransferArray(inStream, outStream, 1);
+
+                    break;
+
+                case PortableUtils.TypeArrayShort:
+                    TransferArray(inStream, outStream, 2);
+
+                    break;
+
+                case PortableUtils.TypeArrayInt:
+                    TransferArray(inStream, outStream, 4);
+
+                    break;
+
+                case PortableUtils.TypeArrayLong:
+                    TransferArray(inStream, outStream, 8);
+
+                    break;
+
+                case PortableUtils.TypeArrayFloat:
+                    TransferArray(inStream, outStream, 4);
+
+                    break;
+
+                case PortableUtils.TypeArrayDouble:
+                    TransferArray(inStream, outStream, 8);
+
+                    break;
+
+                case PortableUtils.TypeArrayChar:
+                    TransferArray(inStream, outStream, 2);
+
+                    break;
+
+                case PortableUtils.TypeArrayBool:
+                    TransferArray(inStream, outStream, 1);
+
+                    break;
+
+                case PortableUtils.TypeArrayDecimal:
+                case PortableUtils.TypeArrayString:
+                case PortableUtils.TypeArrayGuid:
+                case PortableUtils.TypeArrayDate:
+                case PortableUtils.TypeArrayEnum:
+                case PortableUtils.TypeArray:
+                    int arrLen = inStream.ReadInt();
+
+                    outStream.WriteInt(arrLen);
+
+                    for (int i = 0; i < arrLen; i++)
+                        Mutate0(ctx, inStream, outStream, false, 0, null);
+
+                    break;
+
+                case PortableUtils.TypeCollection:
+                    int colLen = inStream.ReadInt();
+
+                    outStream.WriteInt(colLen);
+
+                    outStream.WriteByte(inStream.ReadByte());
+
+                    for (int i = 0; i < colLen; i++)
+                        Mutate0(ctx, inStream, outStream, false, 0, EmptyVals);
+
+                    break;
+
+                case PortableUtils.TypeDictionary:
+                    int dictLen = inStream.ReadInt();
+
+                    outStream.WriteInt(dictLen);
+
+                    outStream.WriteByte(inStream.ReadByte());
+
+                    for (int i = 0; i < dictLen; i++)
+                    {
+                        Mutate0(ctx, inStream, outStream, false, 0, EmptyVals);
+                        Mutate0(ctx, inStream, outStream, false, 0, EmptyVals);
+                    }
+
+                    break;
+
+                case PortableUtils.TypeMapEntry:
+                    Mutate0(ctx, inStream, outStream, false, 0, EmptyVals);
+                    Mutate0(ctx, inStream, outStream, false, 0, EmptyVals);
+
+                    break;
+
+                case PortableUtils.TypePortable:
+                    TransferArray(inStream, outStream, 1); // Data array.
+                    TransferBytes(inStream, outStream, 4); // Offset in array.
+
+                    break;
+
+                case PortableUtils.TypeEnum:
+                    TransferBytes(inStream, outStream, 4); // Integer ordinal.
+
+                    break;
+
+                default:
+                    return false;
+            }
+
+            return true;
+        }
+
+        /// <summary>
+        /// Get's metadata field type ID for the given type.
+        /// </summary>
+        /// <param name="type">Type.</param>
+        /// <returns>Type ID.</returns>
+        private static int TypeId(Type type)
+        {
+            int typeId;
+
+            if (TypeIds.TryGetValue(type, out typeId))
+                return typeId;
+            if (type.IsEnum)
+                return PortableUtils.TypeEnum;
+            if (type.IsArray)
+                return type.GetElementType().IsEnum ? PortableUtils.TypeArrayEnum : PortableUtils.TypeArray;
+            PortableCollectionInfo colInfo = PortableCollectionInfo.Info(type);
+
+            return colInfo.IsAny ? colInfo.IsCollection || colInfo.IsGenericCollection ?
+                PortableUtils.TypeCollection : PortableUtils.TypeDictionary : PortableUtils.TypeObject;
+        }
+
+        /// <summary>
+        /// Transfer bytes from one stream to another.
+        /// </summary>
+        /// <param name="inStream">Input stream.</param>
+        /// <param name="outStream">Output stream.</param>
+        /// <param name="cnt">Bytes count.</param>
+        private static void TransferBytes(PortableHeapStream inStream, IPortableStream outStream, int cnt)
+        {
+            outStream.Write(inStream.InternalArray, inStream.Position, cnt);
+
+            inStream.Seek(cnt, SeekOrigin.Current);
+        }
+
+        /// <summary>
+        /// Transfer array of fixed-size elements from one stream to another.
+        /// </summary>
+        /// <param name="inStream">Input stream.</param>
+        /// <param name="outStream">Output stream.</param>
+        /// <param name="elemSize">Element size.</param>
+        private static void TransferArray(PortableHeapStream inStream, IPortableStream outStream,
+            int elemSize)
+        {
+            int len = inStream.ReadInt();
+
+            outStream.WriteInt(len);
+
+            TransferBytes(inStream, outStream, elemSize * len);
+        }
+
+        /// <summary>
+        /// Mutation ocntext.
+        /// </summary>
+        private class Context
+        {
+            /** Map from object position in old portable to position in new portable. */
+            private IDictionary<int, int> _oldToNew;
+
+            /** Parent context. */
+            private readonly Context _parent;
+
+            /** Portable writer. */
+            private readonly PortableWriterImpl _writer;
+
+            /** Children contexts. */
+            private ICollection<Context> _children;
+
+            /** Closed flag; if context is closed, it can no longer be used. */
+            private bool _closed;
+
+            /// <summary>
+            /// Constructor for parent context where writer invocation is not expected.
+            /// </summary>
+            public Context()
+            {
+                // No-op.
+            }
+
+            /// <summary>
+            /// Constructor for parent context.
+            /// </summary>
+            /// <param name="writer">Writer</param>
+            public Context(PortableWriterImpl writer)
+            {
+                _writer = writer;
+            }
+
+            /// <summary>
+            /// Constructor.
+            /// </summary>
+            /// <param name="parent">Parent context.</param>
+            public Context(Context parent)
+            {
+                _parent = parent;
+                
+                _writer = parent._writer;
+
+                if (parent._children == null)
+                    parent._children = new List<Context>();
+
+                parent._children.Add(this);
+            }
+
+            /// <summary>
+            /// Add another old-to-new position mapping.
+            /// </summary>
+            /// <param name="oldPos">Old position.</param>
+            /// <param name="newPos">New position.</param>
+            /// <param name="hndPos">Handle position.</param>
+            /// <returns><c>True</c> if ampping was added, <c>false</c> if mapping already existed and handle
+            /// position in the new object is returned.</returns>
+            public bool AddOldToNew(int oldPos, int newPos, out int hndPos)
+            {
+                if (_oldToNew == null)
+                    _oldToNew = new Dictionary<int, int>();
+
+                if (_oldToNew.TryGetValue(oldPos, out hndPos))
+                    return false;
+                _oldToNew[oldPos] = newPos;
+
+                return true;
+            }
+
+            /// <summary>
+            /// Get mapping of old position to the new one.
+            /// </summary>
+            /// <param name="oldPos">Old position.</param>
+            /// <param name="newPos">New position.</param>
+            /// <returns><c>True</c> if mapping exists.</returns>
+            public bool OldToNew(int oldPos, out int newPos)
+            {
+                return _oldToNew.TryGetValue(oldPos, out newPos);
+            }
+
+            /// <summary>
+            /// Writer.
+            /// </summary>
+            public PortableWriterImpl Writer
+            {
+                get { return _writer; }
+            }
+
+            /// <summary>
+            /// Closed flag.
+            /// </summary>
+            public bool Closed
+            {
+                get
+                {
+                    return _closed;
+                }
+                set
+                {
+                    Context ctx = this;
+
+                    while (ctx != null)
+                    {
+                        ctx._closed = value;
+
+                        if (_children != null) {
+                            foreach (Context child in _children)
+                                child.Closed = value;
+                        }
+
+                        ctx = ctx._parent;
+                    }
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableCollectionInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableCollectionInfo.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableCollectionInfo.cs
new file mode 100644
index 0000000..fc61833
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableCollectionInfo.cs
@@ -0,0 +1,251 @@
+/*
+ * 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.Portable
+{
+    using System;
+    using System.Collections.Concurrent;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using System.Reflection;
+    using Apache.Ignite.Core.Impl.Common;
+
+    /**
+     * <summary>Collection info helper.</summary>
+     */
+    internal class PortableCollectionInfo
+    {
+        /** Flag: none. */
+        private const byte FlagNone = 0;
+
+        /** Flag: generic dictionary. */
+        private const byte FlagGenericDictionary = 1;
+
+        /** Flag: generic collection. */
+        private const byte FlagGenericCollection = 2;
+
+        /** Flag: dictionary. */
+        private const byte FlagDictionary = 3;
+
+        /** Flag: collection. */
+        private const byte FlagCollection = 4;
+
+        /** Cache "none" value. */
+        private static readonly PortableCollectionInfo None =
+            new PortableCollectionInfo(FlagNone, null, null, null);
+
+        /** Cache "dictionary" value. */
+        private static readonly PortableCollectionInfo Dictionary =
+            new PortableCollectionInfo(FlagDictionary, PortableSystemHandlers.WriteHndDictionary, null, null);
+
+        /** Cache "collection" value. */
+        private static readonly PortableCollectionInfo Collection =
+            new PortableCollectionInfo(FlagCollection, PortableSystemHandlers.WriteHndCollection, null, null);
+
+        /** Cached infos. */
+        private static readonly IDictionary<Type, PortableCollectionInfo> Infos =
+            new ConcurrentDictionary<Type, PortableCollectionInfo>(64, 32);
+
+        /**
+         * <summary>Get collection info for type.</summary>
+         * <param name="type">Type.</param>
+         * <returns>Collection info.</returns>
+         */
+        public static PortableCollectionInfo Info(Type type)
+        {
+            PortableCollectionInfo info;
+
+            if (!Infos.TryGetValue(type, out info))
+            {
+                info = Info0(type);
+
+                Infos[type] = info;
+            }
+
+            return info;
+        }
+
+        /**
+         * <summary>Internal routine to get collection info for type.</summary>
+         * <param name="type">Type.</param>
+         * <returns>Collection info.</returns>
+         */
+        private static PortableCollectionInfo Info0(Type type)
+        {
+            if (type.IsGenericType)
+            {
+                if (type.GetGenericTypeDefinition() == PortableUtils.TypGenericDictionary)
+                {
+                    MethodInfo writeMthd =
+                        PortableUtils.MtdhWriteGenericDictionary.MakeGenericMethod(type.GetGenericArguments());
+                    MethodInfo readMthd =
+                        PortableUtils.MtdhReadGenericDictionary.MakeGenericMethod(type.GetGenericArguments());
+
+                    return new PortableCollectionInfo(FlagGenericDictionary,
+                        PortableSystemHandlers.WriteHndGenericDictionary, writeMthd, readMthd);
+                }
+
+                Type genTyp = type.GetInterface(PortableUtils.TypGenericDictionary.FullName);
+
+                if (genTyp != null)
+                {
+                    MethodInfo writeMthd =
+                        PortableUtils.MtdhWriteGenericDictionary.MakeGenericMethod(genTyp.GetGenericArguments());
+                    MethodInfo readMthd =
+                        PortableUtils.MtdhReadGenericDictionary.MakeGenericMethod(genTyp.GetGenericArguments());
+
+                    return new PortableCollectionInfo(FlagGenericDictionary,
+                        PortableSystemHandlers.WriteHndGenericDictionary, writeMthd, readMthd);
+                }
+
+                if (type.GetGenericTypeDefinition() == PortableUtils.TypGenericCollection)
+                {
+                    MethodInfo writeMthd =
+                        PortableUtils.MtdhWriteGenericCollection.MakeGenericMethod(type.GetGenericArguments());
+                    MethodInfo readMthd =
+                        PortableUtils.MtdhReadGenericCollection.MakeGenericMethod(type.GetGenericArguments());
+
+                    return new PortableCollectionInfo(FlagGenericCollection,
+                        PortableSystemHandlers.WriteHndGenericCollection, writeMthd, readMthd);
+                }
+
+                genTyp = type.GetInterface(PortableUtils.TypGenericCollection.FullName);
+
+                if (genTyp != null)
+                {
+                    MethodInfo writeMthd =
+                        PortableUtils.MtdhWriteGenericCollection.MakeGenericMethod(genTyp.GetGenericArguments());
+                    MethodInfo readMthd =
+                        PortableUtils.MtdhReadGenericCollection.MakeGenericMethod(genTyp.GetGenericArguments());
+
+                    return new PortableCollectionInfo(FlagGenericCollection,
+                        PortableSystemHandlers.WriteHndGenericCollection, writeMthd, readMthd);
+                }
+            }
+
+            if (type == PortableUtils.TypDictionary || type.GetInterface(PortableUtils.TypDictionary.FullName) != null)
+                return Dictionary;
+            if (type == PortableUtils.TypCollection || type.GetInterface(PortableUtils.TypCollection.FullName) != null)
+                return Collection;
+            return None;
+        }
+
+        /** Flag. */
+        private readonly byte _flag;
+
+        /** Write handler. */
+        private readonly PortableSystemWriteDelegate _writeHnd;
+
+        /** Generic write func. */
+        private readonly Action<object, PortableWriterImpl> _writeFunc;
+
+        /** Generic read func. */
+        private readonly Func<PortableReaderImpl, object, object> _readFunc;
+
+        /**
+         * <summary>Constructor.</summary>
+         * <param name="flag0">Flag.</param>
+         * <param name="writeHnd0">Write handler.</param>
+         * <param name="writeMthd0">Generic write method.</param>
+         * <param name="readMthd0">Generic read method.</param>
+         */
+        private PortableCollectionInfo(byte flag0, PortableSystemWriteDelegate writeHnd0,
+            MethodInfo writeMthd0, MethodInfo readMthd0)
+        {
+            _flag = flag0;
+            _writeHnd = writeHnd0;
+
+            if (writeMthd0 != null)
+                _writeFunc = DelegateConverter.CompileFunc<Action<object, PortableWriterImpl>>(null, writeMthd0, null,
+                    new[] {true, false, false});
+
+            if (readMthd0 != null)
+                _readFunc = DelegateConverter.CompileFunc<Func<PortableReaderImpl, object, object>>(null, readMthd0, 
+                    null, new[] {false, true, false});
+        }
+
+        /**
+         * <summary>Generic dictionary flag.</summary>
+         */
+        public bool IsGenericDictionary
+        {
+            get { return _flag == FlagGenericDictionary; }
+        }
+
+        /**
+         * <summary>Generic collection flag.</summary>
+         */
+        public bool IsGenericCollection
+        {
+            get { return _flag == FlagGenericCollection; }
+        }
+
+        /**
+         * <summary>Dictionary flag.</summary>
+         */
+        public bool IsDictionary
+        {
+            get { return _flag == FlagDictionary; }
+        }
+
+        /**
+         * <summary>Collection flag.</summary>
+         */
+        public bool IsCollection
+        {
+            get { return _flag == FlagCollection; }
+        }
+
+        /**
+         * <summary>Whether at least one flag is set..</summary>
+         */
+        public bool IsAny
+        {
+            get { return _flag != FlagNone; }
+        }
+
+        /**
+         * <summary>Write handler.</summary>
+         */
+        public PortableSystemWriteDelegate WriteHandler
+        {
+            get { return _writeHnd; }
+        }
+
+        /// <summary>
+        /// Reads the generic collection.
+        /// </summary>
+        public object ReadGeneric(PortableReaderImpl reader)
+        {
+            Debug.Assert(reader != null);
+            Debug.Assert(_readFunc != null);
+
+            return _readFunc(reader, null);
+        }
+
+        /// <summary>
+        /// Writes the generic collection.
+        /// </summary>
+        public void WriteGeneric(PortableWriterImpl writer, object value)
+        {
+            Debug.Assert(writer != null);
+            Debug.Assert(_writeFunc != null);
+
+            _writeFunc(value, writer);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableFullTypeDescriptor.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableFullTypeDescriptor.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableFullTypeDescriptor.cs
new file mode 100644
index 0000000..f294cbd
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableFullTypeDescriptor.cs
@@ -0,0 +1,203 @@
+/*
+ * 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.Portable
+{
+    using System;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Full type descriptor.
+    /// </summary> 
+    internal class PortableFullTypeDescriptor : IPortableTypeDescriptor
+    {
+        /** Type. */
+        private readonly Type _type;
+
+        /** Type ID. */
+        private readonly int _typeId;
+
+        /** Type name. */
+        private readonly string _typeName;
+
+        /** User type flag. */
+        private readonly bool _userType;
+
+        /** Name converter. */
+        private readonly IPortableNameMapper _nameConverter;
+
+        /** Mapper. */
+        private readonly IPortableIdMapper _mapper;
+
+        /** Serializer. */
+        private readonly IPortableSerializer _serializer;
+
+        /** Metadata enabled flag. */
+        private readonly bool _metaEnabled;
+
+        /** Whether to cache deserialized value in IPortableObject */
+        private readonly bool _keepDeserialized;
+
+        /** Affinity field key name. */
+        private readonly string _affKeyFieldName;
+
+        /** Typed handler. */
+        private readonly object _typedHandler;
+
+        /** Untyped handler. */
+        private readonly PortableSystemWriteDelegate _untypedHandler;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="type">Type.</param>
+        /// <param name="typeId">Type ID.</param>
+        /// <param name="typeName">Type name.</param>
+        /// <param name="userType">User type flag.</param>
+        /// <param name="nameConverter">Name converter.</param>
+        /// <param name="mapper">Mapper.</param>
+        /// <param name="serializer">Serializer.</param>
+        /// <param name="metaEnabled">Metadata enabled flag.</param>
+        /// <param name="keepDeserialized">Whether to cache deserialized value in IPortableObject</param>
+        /// <param name="affKeyFieldName">Affinity field key name.</param>
+        /// <param name="typedHandler">Typed handler.</param>
+        /// <param name="untypedHandler">Untyped handler.</param>
+        public PortableFullTypeDescriptor(
+            Type type, 
+            int typeId, 
+            string typeName, 
+            bool userType, 
+            IPortableNameMapper nameConverter, 
+            IPortableIdMapper mapper, 
+            IPortableSerializer serializer, 
+            bool metaEnabled, 
+            bool keepDeserialized, 
+            string affKeyFieldName, 
+            object typedHandler,
+            PortableSystemWriteDelegate untypedHandler)
+        {
+            _type = type;
+            _typeId = typeId;
+            _typeName = typeName;
+            _userType = userType;
+            _nameConverter = nameConverter;
+            _mapper = mapper;
+            _serializer = serializer;
+            _metaEnabled = metaEnabled;
+            _keepDeserialized = keepDeserialized;
+            _affKeyFieldName = affKeyFieldName;
+            _typedHandler = typedHandler;
+            _untypedHandler = untypedHandler;
+        }
+
+        /// <summary>
+        /// Type.
+        /// </summary>
+        public Type Type
+        {
+            get { return _type; }
+        }
+
+        /// <summary>
+        /// Type ID.
+        /// </summary>
+        public int TypeId
+        {
+            get { return _typeId; }
+        }
+
+        /// <summary>
+        /// Type name.
+        /// </summary>
+        public string TypeName
+        {
+            get { return _typeName; }
+        }
+
+        /// <summary>
+        /// User type flag.
+        /// </summary>
+        public bool UserType
+        {
+            get { return _userType; }
+        }
+
+        /// <summary>
+        /// Metadata enabled flag.
+        /// </summary>
+        public bool MetadataEnabled
+        {
+            get { return _metaEnabled; }
+        }
+
+        /// <summary>
+        /// Whether to cache deserialized value in IPortableObject
+        /// </summary>
+        public bool KeepDeserialized
+        {
+            get { return _keepDeserialized; }
+        }
+
+        /// <summary>
+        /// Name converter.
+        /// </summary>
+        public IPortableNameMapper NameConverter
+        {
+            get { return _nameConverter; }
+        }
+
+        /// <summary>
+        /// Mapper.
+        /// </summary>
+        public IPortableIdMapper Mapper
+        {
+            get { return _mapper; }
+        }
+
+        /// <summary>
+        /// Serializer.
+        /// </summary>
+        public IPortableSerializer Serializer
+        {
+            get { return _serializer; }
+        }
+
+        /// <summary>
+        /// Affinity key field name.
+        /// </summary>
+        public string AffinityKeyFieldName
+        {
+            get { return _affKeyFieldName; }
+        }
+
+        /// <summary>
+        /// Typed handler.
+        /// </summary>
+        public object TypedHandler
+        {
+            get { return _typedHandler; }
+        }
+
+        /// <summary>
+        /// Untyped handler.
+        /// </summary>
+        public PortableSystemWriteDelegate UntypedHandler
+        {
+            get { return _untypedHandler; }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableHandleDictionary.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableHandleDictionary.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableHandleDictionary.cs
new file mode 100644
index 0000000..32e1e02
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableHandleDictionary.cs
@@ -0,0 +1,187 @@
+/*
+ * 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.Portable
+{
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
+
+    /// <summary>
+    /// Object handle dictionary.
+    /// </summary>
+    internal class PortableHandleDictionary<TK, TV>
+    {
+        /** Initial array sizes. */
+        private const int InitialSize = 7;
+
+        /** Dictionary. */
+        private Dictionary<TK, TV> _dict;
+
+        /** First key. */
+        private readonly TK _key1;
+
+        /** First value. */
+        private readonly TV _val1;
+
+        /** Second key. */
+        private TK _key2;
+
+        /** Second value. */
+        private TV _val2;
+
+        /** Third key. */
+        private TK _key3;
+
+        /** Third value. */
+        private TV _val3;
+
+        /// <summary>
+        /// Constructor with initial key-value pair.
+        /// </summary>
+        /// <param name="key">Key.</param>
+        /// <param name="val">Value.</param>
+        [SuppressMessage("Microsoft.Usage", "CA2214:DoNotCallOverridableMethodsInConstructors"),
+         SuppressMessage("ReSharper", "DoNotCallOverridableMethodsInConstructor")]
+        public PortableHandleDictionary(TK key, TV val)
+        {
+            Debug.Assert(!Equals(key, EmptyKey));
+
+            _key1 = key;
+            _val1 = val;
+
+            _key2 = EmptyKey;
+            _key3 = EmptyKey;
+        }
+
+        /// <summary>
+        /// Add value to dictionary.
+        /// </summary>
+        /// <param name="key">Key.</param>
+        /// <param name="val">Value.</param>
+        public void Add(TK key, TV val)
+        {
+            Debug.Assert(!Equals(key, EmptyKey));
+
+            if (Equals(_key2, EmptyKey))
+            {
+                _key2 = key;
+                _val2 = val;
+
+                return;
+            }
+
+            if (Equals(_key3, EmptyKey))
+            {
+                _key3 = key;
+                _val3 = val;
+
+                return;
+            }
+
+            if (_dict == null)
+                _dict = new Dictionary<TK, TV>(InitialSize);
+
+            _dict[key] = val;
+        }
+
+        /// <summary>
+        /// Try getting value for the given key.
+        /// </summary>
+        /// <param name="key">Key.</param>
+        /// <param name="val">Value.</param>
+        /// <returns>True if key was found.</returns>
+        public bool TryGetValue(TK key, out TV val)
+        {
+            Debug.Assert(!Equals(key, EmptyKey));
+
+            if (Equals(key, _key1))
+            {
+                val = _val1;
+
+                return true;
+            }
+
+            if (Equals(key, _key2))
+            {
+                val = _val2;
+
+                return true;
+            }
+
+            if (Equals(key, _key3))
+            {
+                val = _val3;
+
+                return true;
+            }
+
+            if (_dict == null)
+            {
+                val = default(TV);
+
+                return false;
+            }
+
+            return _dict.TryGetValue(key, out val);
+        }
+
+        /// <summary>
+        /// Merge data from another dictionary without overwrite.
+        /// </summary>
+        /// <param name="that">Other dictionary.</param>
+        public void Merge(PortableHandleDictionary<TK, TV> that)
+        {
+            Debug.Assert(that != null, "that == null");
+            
+            AddIfAbsent(that._key1, that._val1);
+            AddIfAbsent(that._key2, that._val2);
+            AddIfAbsent(that._key3, that._val3);
+
+            if (that._dict == null)
+                return;
+
+            foreach (var pair in that._dict)
+                AddIfAbsent(pair.Key, pair.Value);
+        }
+
+        /// <summary>
+        /// Add key/value pair to the bucket if absent.
+        /// </summary>
+        /// <param name="key">Key.</param>
+        /// <param name="val">Value.</param>
+        private void AddIfAbsent(TK key, TV val)
+        {
+            if (Equals(key, EmptyKey))
+                return;
+
+            if (Equals(key, _key1) || Equals(key, _key2) || Equals(key, _key3))
+                return;
+
+            if (_dict == null || !_dict.ContainsKey(key))
+                Add(key, val);
+        }
+
+        /// <summary>
+        /// Gets the empty key.
+        /// </summary>
+        protected virtual TK EmptyKey
+        {
+            get { return default(TK); }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableMarshalAwareSerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableMarshalAwareSerializer.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableMarshalAwareSerializer.cs
new file mode 100644
index 0000000..e3c7523
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableMarshalAwareSerializer.cs
@@ -0,0 +1,45 @@
+/*
+ * 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.Portable
+{
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Portable serializer which only supports <see cref="IPortableMarshalAware"/> types with a default ctor.
+    /// Does not use reflection.
+    /// </summary>
+    internal class PortableMarshalAwareSerializer : IPortableSerializer
+    {
+        /// <summary>
+        /// Default instance.
+        /// </summary>
+        public static readonly PortableMarshalAwareSerializer Instance = new PortableMarshalAwareSerializer();
+
+        /** <inheritdoc /> */
+        public void WritePortable(object obj, IPortableWriter writer)
+        {
+            ((IPortableMarshalAware)obj).WritePortable(writer);
+        }
+
+        /** <inheritdoc /> */
+        public void ReadPortable(object obj, IPortableReader reader)
+        {
+            ((IPortableMarshalAware)obj).ReadPortable(reader);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableMarshaller.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableMarshaller.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableMarshaller.cs
new file mode 100644
index 0000000..4b933a0
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableMarshaller.cs
@@ -0,0 +1,603 @@
+/*
+ * 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.Portable
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Linq;
+    using Apache.Ignite.Core.Impl.Cache;
+    using Apache.Ignite.Core.Impl.Cache.Query.Continuous;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Compute;
+    using Apache.Ignite.Core.Impl.Compute.Closure;
+    using Apache.Ignite.Core.Impl.Datastream;
+    using Apache.Ignite.Core.Impl.Interop;
+    using Apache.Ignite.Core.Impl.Messaging;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Impl.Portable.Metadata;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Portable marshaller implementation.
+    /// </summary>
+    internal class PortableMarshaller
+    {
+        /** Portable configuration. */
+        private readonly PortableConfiguration _cfg;
+
+        /** Type to descriptor map. */
+        private readonly IDictionary<Type, IPortableTypeDescriptor> _typeToDesc =
+            new Dictionary<Type, IPortableTypeDescriptor>();
+
+        /** Type name to descriptor map. */
+        private readonly IDictionary<string, IPortableTypeDescriptor> _typeNameToDesc =
+            new Dictionary<string, IPortableTypeDescriptor>();
+
+        /** ID to descriptor map. */
+        private readonly IDictionary<long, IPortableTypeDescriptor> _idToDesc =
+            new Dictionary<long, IPortableTypeDescriptor>();
+
+        /** Cached metadatas. */
+        private volatile IDictionary<int, PortableMetadataHolder> _metas =
+            new Dictionary<int, PortableMetadataHolder>();
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="cfg">Configurtaion.</param>
+        public PortableMarshaller(PortableConfiguration cfg)
+        {
+            // Validation.
+            if (cfg == null)
+                cfg = new PortableConfiguration();
+
+            if (cfg.TypeConfigurations == null)
+                cfg.TypeConfigurations = new List<PortableTypeConfiguration>();
+
+            foreach (PortableTypeConfiguration typeCfg in cfg.TypeConfigurations)
+            {
+                if (string.IsNullOrEmpty(typeCfg.TypeName))
+                    throw new PortableException("Type name cannot be null or empty: " + typeCfg);
+
+                if (typeCfg.AssemblyName != null && typeCfg.AssemblyName.Length == 0)
+                    throw new PortableException("Assembly name cannot be empty string: " + typeCfg);
+            }
+
+            // Define predefined types.
+            AddPredefinedType(typeof(bool), PortableUtils.TypeBool, PortableSystemHandlers.WriteHndBoolTyped, PortableSystemHandlers.WriteHndBool);
+            AddPredefinedType(typeof(byte), PortableUtils.TypeByte, PortableSystemHandlers.WriteHndByteTyped, PortableSystemHandlers.WriteHndByte);
+            AddPredefinedType(typeof(short), PortableUtils.TypeShort, PortableSystemHandlers.WriteHndShortTyped, PortableSystemHandlers.WriteHndShort);
+            AddPredefinedType(typeof(char), PortableUtils.TypeChar, PortableSystemHandlers.WriteHndCharTyped, PortableSystemHandlers.WriteHndChar);
+            AddPredefinedType(typeof(int), PortableUtils.TypeInt, PortableSystemHandlers.WriteHndIntTyped, PortableSystemHandlers.WriteHndInt);
+            AddPredefinedType(typeof(long), PortableUtils.TypeLong, PortableSystemHandlers.WriteHndLongTyped, PortableSystemHandlers.WriteHndLong);
+            AddPredefinedType(typeof(float), PortableUtils.TypeFloat, PortableSystemHandlers.WriteHndFloatTyped, PortableSystemHandlers.WriteHndFloat);
+            AddPredefinedType(typeof(double), PortableUtils.TypeDouble, PortableSystemHandlers.WriteHndDoubleTyped, PortableSystemHandlers.WriteHndDouble);
+            AddPredefinedType(typeof(string), PortableUtils.TypeString, PortableSystemHandlers.WriteHndStringTyped, PortableSystemHandlers.WriteHndString);
+            AddPredefinedType(typeof(decimal), PortableUtils.TypeDecimal, PortableSystemHandlers.WriteHndDecimalTyped, PortableSystemHandlers.WriteHndDecimal);
+            AddPredefinedType(typeof(DateTime), PortableUtils.TypeDate, PortableSystemHandlers.WriteHndDateTyped, PortableSystemHandlers.WriteHndDate);
+            AddPredefinedType(typeof(Guid), PortableUtils.TypeGuid, PortableSystemHandlers.WriteHndGuidTyped, PortableSystemHandlers.WriteHndGuid);
+
+            // TODO: Remove this registration
+            AddPredefinedType(typeof(PortableUserObject), PortableUtils.TypePortable, PortableSystemHandlers.WriteHndPortableTyped, 
+                PortableSystemHandlers.WriteHndPortable);
+
+            AddPredefinedType(typeof(bool[]), PortableUtils.TypeArrayBool, PortableSystemHandlers.WriteHndBoolArrayTyped,
+                PortableSystemHandlers.WriteHndBoolArray);
+            AddPredefinedType(typeof(byte[]), PortableUtils.TypeArrayByte, PortableSystemHandlers.WriteHndByteArrayTyped,
+                PortableSystemHandlers.WriteHndByteArray);
+            AddPredefinedType(typeof(short[]), PortableUtils.TypeArrayShort, PortableSystemHandlers.WriteHndShortArrayTyped,
+                PortableSystemHandlers.WriteHndShortArray);
+            AddPredefinedType(typeof(char[]), PortableUtils.TypeArrayChar, PortableSystemHandlers.WriteHndCharArrayTyped,
+                PortableSystemHandlers.WriteHndCharArray);
+            AddPredefinedType(typeof(int[]), PortableUtils.TypeArrayInt, PortableSystemHandlers.WriteHndIntArrayTyped,
+                PortableSystemHandlers.WriteHndIntArray);
+            AddPredefinedType(typeof(long[]), PortableUtils.TypeArrayLong, PortableSystemHandlers.WriteHndLongArrayTyped,
+                PortableSystemHandlers.WriteHndLongArray);
+            AddPredefinedType(typeof(float[]), PortableUtils.TypeArrayFloat, PortableSystemHandlers.WriteHndFloatArrayTyped,
+                PortableSystemHandlers.WriteHndFloatArray);
+            AddPredefinedType(typeof(double[]), PortableUtils.TypeArrayDouble, PortableSystemHandlers.WriteHndDoubleArrayTyped,
+                PortableSystemHandlers.WriteHndDoubleArray);
+            AddPredefinedType(typeof(decimal[]), PortableUtils.TypeArrayDecimal, PortableSystemHandlers.WriteHndDecimalArrayTyped,
+                PortableSystemHandlers.WriteHndDecimalArray);
+            AddPredefinedType(typeof(string[]), PortableUtils.TypeArrayString, PortableSystemHandlers.WriteHndStringArrayTyped,
+                PortableSystemHandlers.WriteHndStringArray);
+            AddPredefinedType(typeof(DateTime?[]), PortableUtils.TypeArrayDate, PortableSystemHandlers.WriteHndDateArrayTyped,
+                PortableSystemHandlers.WriteHndDateArray);
+            AddPredefinedType(typeof(Guid?[]), PortableUtils.TypeArrayGuid, PortableSystemHandlers.WriteHndGuidArrayTyped,
+                PortableSystemHandlers.WriteHndGuidArray);
+
+            // Define system types. They use internal reflective stuff, so configuration doesn't affect them.
+            AddSystemTypes();
+
+            // 2. Define user types.
+            var dfltSerializer = cfg.DefaultSerializer == null ? new PortableReflectiveSerializer() : null;
+
+            var typeResolver = new TypeResolver();
+
+            ICollection<PortableTypeConfiguration> typeCfgs = cfg.TypeConfigurations;
+
+            if (typeCfgs != null)
+                foreach (PortableTypeConfiguration typeCfg in typeCfgs)
+                    AddUserType(cfg, typeCfg, typeResolver, dfltSerializer);
+
+            ICollection<string> types = cfg.Types;
+
+            if (types != null)
+                foreach (string type in types)
+                    AddUserType(cfg, new PortableTypeConfiguration(type), typeResolver, dfltSerializer);
+
+            if (cfg.DefaultSerializer == null)
+                cfg.DefaultSerializer = dfltSerializer;
+
+            _cfg = cfg;
+        }
+
+        /// <summary>
+        /// Gets or sets the backing grid.
+        /// </summary>
+        public Ignite Ignite { get; set; }
+
+        /// <summary>
+        /// Marshal object.
+        /// </summary>
+        /// <param name="val">Value.</param>
+        /// <returns>Serialized data as byte array.</returns>
+        public byte[] Marshal(object val)
+        {
+            PortableHeapStream stream = new PortableHeapStream(128);
+
+            Marshal(val, stream);
+
+            return stream.ArrayCopy();
+        }
+
+        /// <summary>
+        /// Marshal object.
+        /// </summary>
+        /// <param name="val">Value.</param>
+        /// <param name="stream">Output stream.</param>
+        /// <returns>Collection of metadatas (if any).</returns>
+        private void Marshal<T>(T val, IPortableStream stream)
+        {
+            PortableWriterImpl writer = StartMarshal(stream);
+
+            writer.Write(val);
+
+            FinishMarshal(writer);
+        }
+
+        /// <summary>
+        /// Start marshal session.
+        /// </summary>
+        /// <param name="stream">Stream.</param>
+        /// <returns>Writer.</returns>
+        public PortableWriterImpl StartMarshal(IPortableStream stream)
+        {
+            return new PortableWriterImpl(this, stream);
+        }
+
+        /// <summary>
+        /// Finish marshal session.
+        /// </summary>
+        /// <param name="writer">Writer.</param>
+        /// <returns>Dictionary with metadata.</returns>
+        public void FinishMarshal(IPortableWriter writer)
+        {
+            var meta = ((PortableWriterImpl) writer).Metadata();
+
+            var ignite = Ignite;
+
+            if (ignite != null && meta != null && meta.Count > 0)
+                ignite.PutMetadata(meta);
+        }
+
+        /// <summary>
+        /// Unmarshal object.
+        /// </summary>
+        /// <typeparam name="T"></typeparam>
+        /// <param name="data">Data array.</param>
+        /// <param name="keepPortable">Whether to keep portables as portables.</param>
+        /// <returns>
+        /// Object.
+        /// </returns>
+        public T Unmarshal<T>(byte[] data, bool keepPortable)
+        {
+            return Unmarshal<T>(new PortableHeapStream(data), keepPortable);
+        }
+
+        /// <summary>
+        /// Unmarshal object.
+        /// </summary>
+        /// <param name="data">Data array.</param>
+        /// <param name="mode">The mode.</param>
+        /// <returns>
+        /// Object.
+        /// </returns>
+        public T Unmarshal<T>(byte[] data, PortableMode mode = PortableMode.Deserialize)
+        {
+            return Unmarshal<T>(new PortableHeapStream(data), mode);
+        }
+
+        /// <summary>
+        /// Unmarshal object.
+        /// </summary>
+        /// <param name="stream">Stream over underlying byte array with correct position.</param>
+        /// <param name="keepPortable">Whether to keep portables as portables.</param>
+        /// <returns>
+        /// Object.
+        /// </returns>
+        public T Unmarshal<T>(IPortableStream stream, bool keepPortable)
+        {
+            return Unmarshal<T>(stream, keepPortable ? PortableMode.KeepPortable : PortableMode.Deserialize, null);
+        }
+
+        /// <summary>
+        /// Unmarshal object.
+        /// </summary>
+        /// <param name="stream">Stream over underlying byte array with correct position.</param>
+        /// <param name="mode">The mode.</param>
+        /// <returns>
+        /// Object.
+        /// </returns>
+        public T Unmarshal<T>(IPortableStream stream, PortableMode mode = PortableMode.Deserialize)
+        {
+            return Unmarshal<T>(stream, mode, null);
+        }
+
+        /// <summary>
+        /// Unmarshal object.
+        /// </summary>
+        /// <param name="stream">Stream over underlying byte array with correct position.</param>
+        /// <param name="mode">The mode.</param>
+        /// <param name="builder">Builder.</param>
+        /// <returns>
+        /// Object.
+        /// </returns>
+        public T Unmarshal<T>(IPortableStream stream, PortableMode mode, PortableBuilderImpl builder)
+        {
+            return new PortableReaderImpl(this, _idToDesc, stream, mode, builder).Deserialize<T>();
+        }
+
+        /// <summary>
+        /// Start unmarshal session.
+        /// </summary>
+        /// <param name="stream">Stream.</param>
+        /// <param name="keepPortable">Whether to keep portables as portables.</param>
+        /// <returns>
+        /// Reader.
+        /// </returns>
+        public PortableReaderImpl StartUnmarshal(IPortableStream stream, bool keepPortable)
+        {
+            return new PortableReaderImpl(this, _idToDesc, stream,
+                keepPortable ? PortableMode.KeepPortable : PortableMode.Deserialize, null);
+        }
+
+        /// <summary>
+        /// Start unmarshal session.
+        /// </summary>
+        /// <param name="stream">Stream.</param>
+        /// <param name="mode">The mode.</param>
+        /// <returns>Reader.</returns>
+        public PortableReaderImpl StartUnmarshal(IPortableStream stream, PortableMode mode = PortableMode.Deserialize)
+        {
+            return new PortableReaderImpl(this, _idToDesc, stream, mode, null);
+        }
+        
+        /// <summary>
+        /// Gets metadata for the given type ID.
+        /// </summary>
+        /// <param name="typeId">Type ID.</param>
+        /// <returns>Metadata or null.</returns>
+        public IPortableMetadata Metadata(int typeId)
+        {
+            if (Ignite != null)
+            {
+                IPortableMetadata meta = Ignite.Metadata(typeId);
+
+                if (meta != null)
+                    return meta;
+            }
+
+            return PortableMetadataImpl.EmptyMeta;
+        }
+
+        /// <summary>
+        /// Gets metadata handler for the given type ID.
+        /// </summary>
+        /// <param name="desc">Type descriptor.</param>
+        /// <returns>Metadata handler.</returns>
+        public IPortableMetadataHandler MetadataHandler(IPortableTypeDescriptor desc)
+        {
+            PortableMetadataHolder holder;
+
+            if (!_metas.TryGetValue(desc.TypeId, out holder))
+            {
+                lock (this)
+                {
+                    if (!_metas.TryGetValue(desc.TypeId, out holder))
+                    {
+                        IDictionary<int, PortableMetadataHolder> metas0 =
+                            new Dictionary<int, PortableMetadataHolder>(_metas);
+
+                        holder = desc.MetadataEnabled ? new PortableMetadataHolder(desc.TypeId,
+                            desc.TypeName, desc.AffinityKeyFieldName) : null;
+
+                        metas0[desc.TypeId] = holder;
+
+                        _metas = metas0;
+                    }
+                }
+            }
+
+            if (holder != null)
+            {
+                ICollection<int> ids = holder.FieldIds();
+
+                bool newType = ids.Count == 0 && !holder.Saved();
+
+                return new PortableHashsetMetadataHandler(ids, newType);
+            }
+            return null;
+        }
+
+        /// <summary>
+        /// Callback invoked when metadata has been sent to the server and acknowledged by it.
+        /// </summary>
+        /// <param name="newMetas"></param>
+        public void OnMetadataSent(IDictionary<int, IPortableMetadata> newMetas)
+        {
+            foreach (KeyValuePair<int, IPortableMetadata> metaEntry in newMetas)
+            {
+                PortableMetadataImpl meta = (PortableMetadataImpl) metaEntry.Value;
+
+                IDictionary<int, Tuple<string, int>> mergeInfo =
+                    new Dictionary<int, Tuple<string, int>>(meta.FieldsMap().Count);
+
+                foreach (KeyValuePair<string, int> fieldMeta in meta.FieldsMap())
+                {
+                    int fieldId = PortableUtils.FieldId(metaEntry.Key, fieldMeta.Key, null, null);
+
+                    mergeInfo[fieldId] = new Tuple<string, int>(fieldMeta.Key, fieldMeta.Value);
+                }
+
+                _metas[metaEntry.Key].Merge(mergeInfo);
+            }
+        }
+        
+        /// <summary>
+        /// Gets descriptor for type.
+        /// </summary>
+        /// <param name="type">Type.</param>
+        /// <returns>Descriptor.</returns>
+        public IPortableTypeDescriptor Descriptor(Type type)
+        {
+            IPortableTypeDescriptor desc;
+
+            _typeToDesc.TryGetValue(type, out desc);
+
+            return desc;
+        }
+
+        /// <summary>
+        /// Gets descriptor for type name.
+        /// </summary>
+        /// <param name="typeName">Type name.</param>
+        /// <returns>Descriptor.</returns>
+        public IPortableTypeDescriptor Descriptor(string typeName)
+        {
+            IPortableTypeDescriptor desc;
+
+            return _typeNameToDesc.TryGetValue(typeName, out desc) ? desc : 
+                new PortableSurrogateTypeDescriptor(_cfg, typeName);
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        /// <param name="userType"></param>
+        /// <param name="typeId"></param>
+        /// <returns></returns>
+        public IPortableTypeDescriptor Descriptor(bool userType, int typeId)
+        {
+            IPortableTypeDescriptor desc;
+
+            return _idToDesc.TryGetValue(PortableUtils.TypeKey(userType, typeId), out desc) ? desc :
+                userType ? new PortableSurrogateTypeDescriptor(_cfg, typeId) : null;
+        }
+
+        /// <summary>
+        /// Add user type.
+        /// </summary>
+        /// <param name="cfg">Configuration.</param>
+        /// <param name="typeCfg">Type configuration.</param>
+        /// <param name="typeResolver">The type resolver.</param>
+        /// <param name="dfltSerializer">The default serializer.</param>
+        private void AddUserType(PortableConfiguration cfg, PortableTypeConfiguration typeCfg, 
+            TypeResolver typeResolver, IPortableSerializer dfltSerializer)
+        {
+            // Get converter/mapper/serializer.
+            IPortableNameMapper nameMapper = typeCfg.NameMapper ?? cfg.DefaultNameMapper;
+
+            IPortableIdMapper idMapper = typeCfg.IdMapper ?? cfg.DefaultIdMapper;
+
+            bool metaEnabled = typeCfg.MetadataEnabled ?? cfg.DefaultMetadataEnabled;
+
+            bool keepDeserialized = typeCfg.KeepDeserialized ?? cfg.DefaultKeepDeserialized;
+
+            // Try resolving type.
+            Type type = typeResolver.ResolveType(typeCfg.TypeName, typeCfg.AssemblyName);
+
+            if (type != null)
+            {
+                // Type is found.
+                var typeName = GetTypeName(type);
+
+                int typeId = PortableUtils.TypeId(typeName, nameMapper, idMapper);
+
+                var serializer = typeCfg.Serializer ?? cfg.DefaultSerializer
+                                 ?? GetPortableMarshalAwareSerializer(type) ?? dfltSerializer;
+
+                var refSerializer = serializer as PortableReflectiveSerializer;
+
+                if (refSerializer != null)
+                    refSerializer.Register(type, typeId, nameMapper, idMapper);
+
+                AddType(type, typeId, typeName, true, metaEnabled, keepDeserialized, nameMapper, idMapper, serializer,
+                    typeCfg.AffinityKeyFieldName, null, null);
+            }
+            else
+            {
+                // Type is not found.
+                string typeName = PortableUtils.SimpleTypeName(typeCfg.TypeName);
+
+                int typeId = PortableUtils.TypeId(typeName, nameMapper, idMapper);
+
+                AddType(null, typeId, typeName, true, metaEnabled, keepDeserialized, nameMapper, idMapper, null,
+                    typeCfg.AffinityKeyFieldName, null, null);
+            }
+        }
+
+        /// <summary>
+        /// Gets the <see cref="PortableMarshalAwareSerializer"/> for a type if it is compatible.
+        /// </summary>
+        /// <param name="type">The type.</param>
+        /// <returns>Resulting <see cref="PortableMarshalAwareSerializer"/>, or null.</returns>
+        private static IPortableSerializer GetPortableMarshalAwareSerializer(Type type)
+        {
+            return type.GetInterfaces().Contains(typeof (IPortableMarshalAware)) 
+                ? PortableMarshalAwareSerializer.Instance 
+                : null;
+        }
+
+        /// <summary>
+        /// Add predefined type.
+        /// </summary>
+        /// <param name="type">Type.</param>
+        /// <param name="typeId">Type ID.</param>
+        /// <param name="typedHandler">Typed handler.</param>
+        /// <param name="untypedHandler">Untyped handler.</param>
+        private void AddPredefinedType(Type type, int typeId, object typedHandler,
+            PortableSystemWriteDelegate untypedHandler)
+        {
+            AddType(type, typeId, GetTypeName(type), false, false, false, null, null, null, null, typedHandler,
+                untypedHandler);
+        }
+
+        /// <summary>
+        /// Add type.
+        /// </summary>
+        /// <param name="type">Type.</param>
+        /// <param name="typeId">Type ID.</param>
+        /// <param name="typeName">Type name.</param>
+        /// <param name="userType">User type flag.</param>
+        /// <param name="metaEnabled">Metadata enabled flag.</param>
+        /// <param name="keepDeserialized">Whether to cache deserialized value in IPortableObject</param>
+        /// <param name="nameMapper">Name mapper.</param>
+        /// <param name="idMapper">ID mapper.</param>
+        /// <param name="serializer">Serializer.</param>
+        /// <param name="affKeyFieldName">Affinity key field name.</param>
+        /// <param name="typedHandler">Typed handler.</param>
+        /// <param name="untypedHandler">Untyped handler.</param>
+        private void AddType(Type type, int typeId, string typeName, bool userType, bool metaEnabled,
+            bool keepDeserialized, IPortableNameMapper nameMapper, IPortableIdMapper idMapper,
+            IPortableSerializer serializer, string affKeyFieldName, object typedHandler,
+            PortableSystemWriteDelegate untypedHandler)
+        {
+            long typeKey = PortableUtils.TypeKey(userType, typeId);
+
+            if (_idToDesc.ContainsKey(typeKey))
+            {
+                string type1 = _idToDesc[typeKey].Type != null ? _idToDesc[typeKey].Type.AssemblyQualifiedName : null;
+                string type2 = type != null ? type.AssemblyQualifiedName : null;
+
+                throw new PortableException("Conflicting type IDs [type1=" + type1 + ", type2=" + type2 +
+                    ", typeId=" + typeId + ']');
+            }
+
+            if (userType && _typeNameToDesc.ContainsKey(typeName))
+                throw new PortableException("Conflicting type name: " + typeName);
+
+            IPortableTypeDescriptor descriptor =
+                new PortableFullTypeDescriptor(type, typeId, typeName, userType, nameMapper, idMapper, serializer,
+                    metaEnabled, keepDeserialized, affKeyFieldName, typedHandler, untypedHandler);
+
+            if (type != null)
+                _typeToDesc[type] = descriptor;
+
+            if (userType)
+                _typeNameToDesc[typeName] = descriptor;
+
+            _idToDesc[typeKey] = descriptor;            
+        }
+
+        /// <summary>
+        /// Adds a predefined system type.
+        /// </summary>
+        private void AddSystemType<T>(byte typeId, Func<PortableReaderImpl, T> ctor) where T : IPortableWriteAware
+        {
+            var type = typeof(T);
+
+            var serializer = new PortableSystemTypeSerializer<T>(ctor);
+
+            AddType(type, typeId, GetTypeName(type), false, false, false, null, null, serializer, null, null, null);
+        }
+
+        /// <summary>
+        /// Adds predefined system types.
+        /// </summary>
+        private void AddSystemTypes()
+        {
+            AddSystemType(PortableUtils.TypeNativeJobHolder, w => new ComputeJobHolder(w));
+            AddSystemType(PortableUtils.TypeComputeJobWrapper, w => new ComputeJobWrapper(w));
+            AddSystemType(PortableUtils.TypePortableJobResHolder, w => new PortableResultWrapper(w));
+            AddSystemType(PortableUtils.TypeDotNetCfg, w => new InteropDotNetConfiguration(w));
+            AddSystemType(PortableUtils.TypeDotNetPortableCfg, w => new InteropDotNetPortableConfiguration(w));
+            AddSystemType(PortableUtils.TypeDotNetPortableTypCfg, w => new InteropDotNetPortableTypeConfiguration(w));
+            AddSystemType(PortableUtils.TypeIgniteProxy, w => new IgniteProxy());
+            AddSystemType(PortableUtils.TypeComputeOutFuncJob, w => new ComputeOutFuncJob(w));
+            AddSystemType(PortableUtils.TypeComputeOutFuncWrapper, w => new ComputeOutFuncWrapper(w));
+            AddSystemType(PortableUtils.TypeComputeFuncWrapper, w => new ComputeFuncWrapper(w));
+            AddSystemType(PortableUtils.TypeComputeFuncJob, w => new ComputeFuncJob(w));
+            AddSystemType(PortableUtils.TypeComputeActionJob, w => new ComputeActionJob(w));
+            AddSystemType(PortableUtils.TypeContinuousQueryRemoteFilterHolder, w => new ContinuousQueryFilterHolder(w));
+            AddSystemType(PortableUtils.TypeSerializableHolder, w => new SerializableObjectHolder(w));
+            AddSystemType(PortableUtils.TypeCacheEntryProcessorHolder, w => new CacheEntryProcessorHolder(w));
+            AddSystemType(PortableUtils.TypeCacheEntryPredicateHolder, w => new CacheEntryFilterHolder(w));
+            AddSystemType(PortableUtils.TypeMessageFilterHolder, w => new MessageFilterHolder(w));
+            AddSystemType(PortableUtils.TypePortableOrSerializableHolder, w => new PortableOrSerializableObjectHolder(w));
+            AddSystemType(PortableUtils.TypeStreamReceiverHolder, w => new StreamReceiverHolder(w));
+        }
+
+        /// <summary>
+        /// Gets the name of the type.
+        /// </summary>
+        /// <param name="type">The type.</param>
+        /// <returns>
+        /// Simple type name for non-generic types; simple type name with appended generic arguments for generic types.
+        /// </returns>
+        private static string GetTypeName(Type type)
+        {
+            if (!type.IsGenericType)
+                return type.Name;
+
+            var args = type.GetGenericArguments().Select(GetTypeName).Aggregate((x, y) => x + "," + y);
+
+            return string.Format("{0}[{1}]", type.Name, args);
+        }
+    }
+}


[16/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableObject.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableObject.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableObject.cs
new file mode 100644
index 0000000..1cedf37
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableObject.cs
@@ -0,0 +1,44 @@
+/*
+ * 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.Portable
+{
+    /// <summary>
+    /// Wrapper for serialized portable objects.
+    /// </summary>
+    public interface IPortableObject
+    {
+        /// <summary>Gets portable object type ID.</summary>
+        /// <returns>Type ID.</returns>
+        int TypeId();
+
+        /// <summary>
+        /// Gets object metadata.
+        /// </summary>
+        /// <returns>Metadata.</returns>
+        IPortableMetadata Metadata();
+
+        /// <summary>Gets field value.</summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Field value.</returns>
+        TF Field<TF>(string fieldName);
+
+        /// <summary>Gets fully deserialized instance of portable object.</summary>
+        /// <returns>Fully deserialized instance of portable object.</returns>
+        T Deserialize<T>();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableRawReader.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableRawReader.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableRawReader.cs
new file mode 100644
index 0000000..ee2520d
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableRawReader.cs
@@ -0,0 +1,264 @@
+/*
+ * 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.Portable
+{
+    using System;
+    using System.Collections;
+    using System.Collections.Generic;
+
+    /// <summary>
+    /// Raw reader for portable objects. 
+    /// </summary>
+    public interface IPortableRawReader
+    {
+        /// <summary>
+        /// Read byte value. 
+        /// </summary>
+        /// <returns>Byte value.</returns>
+        byte ReadByte();
+
+        /// <summary>
+        /// Read byte array. 
+        /// </summary>
+        /// <returns>Byte array.</returns>
+        byte[] ReadByteArray();
+
+        /// <summary>
+        /// Read char value. 
+        /// </summary>
+        /// <returns>Char value.</returns>
+        char ReadChar();
+
+        /// <summary>
+        /// Read char array. 
+        /// </summary>
+        /// <returns>Char array.</returns>
+        char[] ReadCharArray();
+
+        /// <summary>
+        /// Read short value. 
+        /// </summary>
+        /// <returns>Short value.</returns>
+        short ReadShort();
+
+        /// <summary>
+        /// Read short array. 
+        /// </summary>
+        /// <returns>Short array.</returns>
+        short[] ReadShortArray();
+
+        /// <summary>
+        /// Read int value. 
+        /// </summary>
+        /// <returns>Int value.</returns>
+        int ReadInt();
+
+        /// <summary>
+        /// Read int array. 
+        /// </summary>
+        /// <returns>Int array.</returns>
+        int[] ReadIntArray();
+
+        /// <summary>
+        /// Read long value. 
+        /// </summary>
+        /// <returns>Long value.</returns>
+        long ReadLong();
+
+        /// <summary>
+        /// Read long array. 
+        /// </summary>
+        /// <returns>Long array.</returns>
+        long[] ReadLongArray();
+
+        /// <summary>
+        /// Read boolean value. 
+        /// </summary>
+        /// <returns>Boolean value.</returns>
+        bool ReadBoolean();
+
+        /// <summary>
+        /// Read boolean array. 
+        /// </summary>
+        /// <returns>Boolean array.</returns>
+        bool[] ReadBooleanArray();
+
+        /// <summary>
+        /// Read float value. 
+        /// </summary>
+        /// <returns>Float value.</returns>
+        float ReadFloat();
+
+        /// <summary>
+        /// Read float array. 
+        /// </summary>
+        /// <returns>Float array.</returns>
+        float[] ReadFloatArray();
+
+        /// <summary>
+        /// Read double value. 
+        /// </summary>
+        /// <returns>Double value.</returns>
+        double ReadDouble();
+
+        /// <summary>
+        /// Read double array. 
+        /// </summary>
+        /// <returns>Double array.</returns>
+        double[] ReadDoubleArray();
+
+        /// <summary>
+        /// Read decimal value. 
+        /// </summary>
+        /// <returns>Decimal value.</returns>
+        decimal ReadDecimal();
+
+        /// <summary>
+        /// Read decimal array. 
+        /// </summary>
+        /// <returns>Decimal array.</returns>
+        decimal[] ReadDecimalArray();
+
+        /// <summary>
+        /// Read date value in UTC form. Shortcut for <c>ReadDate(false)</c>.
+        /// </summary>
+        /// <returns>Date value.</returns>
+        DateTime? ReadDate();
+
+        /// <summary>
+        /// Read date value.
+        /// </summary>
+        /// <param name="local">Whether to read date in local (<c>true</c>) or UTC (<c>false</c>) form.</param>
+        /// <returns></returns>
+        DateTime? ReadDate(bool local);
+
+        /// <summary>
+        /// Read date array in UTC form. Shortcut for <c>ReadDateArray(false)</c>.
+        /// </summary>
+        /// <returns>Date array.</returns>
+        DateTime?[] ReadDateArray();
+
+        /// <summary>
+        /// Read date array.
+        /// </summary>
+        /// <param name="local">Whether to read date array in local (<c>true</c>) or UTC (<c>false</c>) form.</param>
+        /// <returns>Date array.</returns>
+        DateTime?[] ReadDateArray(bool local);
+
+        /// <summary>
+        /// Read string value. 
+        /// </summary>
+        /// <returns>String value.</returns>
+        string ReadString();
+
+        /// <summary>
+        /// Read string array. 
+        /// </summary>
+        /// <returns>String array.</returns>
+        string[] ReadStringArray();
+
+        /// <summary>
+        /// Read GUID value. 
+        /// </summary>
+        /// <returns>GUID value.</returns>
+        Guid? ReadGuid();
+
+        /// <summary>
+        /// Read GUID array. 
+        /// </summary>
+        /// <returns>GUID array.</returns>
+        Guid?[] ReadGuidArray();
+
+        /// <summary>
+        /// Read enum value.
+        /// </summary>
+        /// <returns>Enum value.</returns>
+        T ReadEnum<T>();
+
+        /// <summary>
+        /// Read enum array.
+        /// </summary>
+        /// <returns>Enum array.</returns>
+        T[] ReadEnumArray<T>();
+        
+        /// <summary>
+        /// Read object. 
+        /// </summary>
+        /// <returns>Object.</returns>
+        T ReadObject<T>();
+
+        /// <summary>
+        /// Read object array. 
+        /// </summary>
+        /// <returns>Object array.</returns>
+        T[] ReadObjectArray<T>();
+
+        /// <summary>
+        /// Read collection. 
+        /// </summary>
+        /// <returns>Collection.</returns>
+        ICollection ReadCollection();
+
+        /// <summary>
+        /// Read collection.
+        /// </summary>
+        /// <param name="factory">Factory.</param>
+        /// <param name="adder">Adder.</param>
+        /// <returns>Collection.</returns>
+        ICollection ReadCollection(PortableCollectionFactory factory, PortableCollectionAdder adder);
+
+        /// <summary>
+        /// Read generic collection. 
+        /// </summary>
+        /// <returns>Collection.</returns>
+        ICollection<T> ReadGenericCollection<T>();
+
+        /// <summary>
+        /// Read generic collection.
+        /// </summary>
+        /// <param name="factory">Factory.</param>
+        /// <returns>Collection.</returns>
+        ICollection<T> ReadGenericCollection<T>(PortableGenericCollectionFactory<T> factory);
+
+        /// <summary>
+        /// Read dictionary. 
+        /// </summary>
+        /// <returns>Dictionary.</returns>
+        IDictionary ReadDictionary();
+
+        /// <summary>
+        /// Read dictionary.
+        /// </summary>
+        /// <param name="factory">Factory.</param>
+        /// <returns>Dictionary.</returns>
+        IDictionary ReadDictionary(PortableDictionaryFactory factory);
+
+        /// <summary>
+        /// Read generic dictionary. 
+        /// </summary>
+        /// <returns>Dictionary.</returns>
+        IDictionary<TK, TV> ReadGenericDictionary<TK, TV>();
+
+        /// <summary>
+        /// Read generic dictionary.
+        /// </summary>
+        /// <param name="factory">Factory.</param>
+        /// <returns>Dictionary.</returns>
+        IDictionary<TK, TV> ReadGenericDictionary<TK, TV>(PortableGenericDictionaryFactory<TK, TV> factory);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableRawWriter.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableRawWriter.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableRawWriter.cs
new file mode 100644
index 0000000..eacfde3
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableRawWriter.cs
@@ -0,0 +1,221 @@
+/*
+ * 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.Portable
+{
+    using System;
+    using System.Collections;
+    using System.Collections.Generic;
+
+    /// <summary>
+    /// Raw writer for portable objects. 
+    /// </summary>
+    public interface IPortableRawWriter
+    {
+        /// <summary>
+        /// Write byte value.
+        /// </summary>
+        /// <param name="val">Byte value.</param>
+        void WriteByte(byte val);
+
+        /// <summary>
+        /// Write byte array.
+        /// </summary>
+        /// <param name="val">Byte array.</param>
+        void WriteByteArray(byte[] val);
+
+        /// <summary>
+        /// Write char value.
+        /// </summary>
+        /// <param name="val">Char value.</param>
+        void WriteChar(char val);
+
+        /// <summary>
+        /// Write char array.
+        /// </summary>
+        /// <param name="val">Char array.</param>
+        void WriteCharArray(char[] val);
+
+        /// <summary>
+        /// Write short value.
+        /// </summary>
+        /// <param name="val">Short value.</param>
+        void WriteShort(short val);
+
+        /// <summary>
+        /// Write short array.
+        /// </summary>
+        /// <param name="val">Short array.</param>
+        void WriteShortArray(short[] val);
+
+        /// <summary>
+        /// Write int value.
+        /// </summary>
+        /// <param name="val">Int value.</param>
+        void WriteInt(int val);
+
+        /// <summary>
+        /// Write int array.
+        /// </summary>
+        /// <param name="val">Int array.</param>
+        void WriteIntArray(int[] val);
+
+        /// <summary>
+        /// Write long value.
+        /// </summary>
+        /// <param name="val">Long value.</param>
+        void WriteLong(long val);
+
+        /// <summary>
+        /// Write long array.
+        /// </summary>
+        /// <param name="val">Long array.</param>
+        void WriteLongArray(long[] val);
+
+        /// <summary>
+        /// Write boolean value.
+        /// </summary>
+        /// <param name="val">Boolean value.</param>
+        void WriteBoolean(bool val);
+
+        /// <summary>
+        /// Write boolean array.
+        /// </summary>
+        /// <param name="val">Boolean array.</param>
+        void WriteBooleanArray(bool[] val);
+
+        /// <summary>
+        /// Write float value.
+        /// </summary>
+        /// <param name="val">Float value.</param>
+        void WriteFloat(float val);
+
+        /// <summary>
+        /// Write float array.
+        /// </summary>
+        /// <param name="val">Float array.</param>
+        void WriteFloatArray(float[] val);
+
+        /// <summary>
+        /// Write double value.
+        /// </summary>
+        /// <param name="val">Double value.</param>
+        void WriteDouble(double val);
+
+        /// <summary>
+        /// Write double array.
+        /// </summary>
+        /// <param name="val">Double array.</param>
+        void WriteDoubleArray(double[] val);
+
+        /// <summary>
+        /// Write decimal value.
+        /// </summary>
+        /// <param name="val">Decimal value.</param>
+        void WriteDecimal(decimal val);
+
+        /// <summary>
+        /// Write decimal array.
+        /// </summary>
+        /// <param name="val">Decimal array.</param>
+        void WriteDecimalArray(decimal[] val);
+
+        /// <summary>
+        /// Write date value.
+        /// </summary>
+        /// <param name="val">Date value.</param>
+        void WriteDate(DateTime? val);
+
+        /// <summary>
+        /// Write date array.
+        /// </summary>
+        /// <param name="val">Date array.</param>
+        void WriteDateArray(DateTime?[] val);
+
+        /// <summary>
+        /// Write string value.
+        /// </summary>
+        /// <param name="val">String value.</param>
+        void WriteString(string val);
+
+        /// <summary>
+        /// Write string array.
+        /// </summary>
+        /// <param name="val">String array.</param>
+        void WriteStringArray(string[] val);
+
+        /// <summary>
+        /// Write GUID value.
+        /// </summary>
+        /// <param name="val">GUID value.</param>
+        void WriteGuid(Guid? val);
+
+        /// <summary>
+        /// Write GUID array.
+        /// </summary>
+        /// <param name="val">GUID array.</param>
+        void WriteGuidArray(Guid?[] val);
+
+        /// <summary>
+        /// Write enum value.
+        /// </summary>
+        /// <param name="val">Enum value.</param>
+        void WriteEnum<T>(T val);
+
+        /// <summary>
+        /// Write enum array.
+        /// </summary>
+        /// <param name="val">Enum array.</param>
+        void WriteEnumArray<T>(T[] val);
+
+        /// <summary>
+        /// Write object value.
+        /// </summary>
+        /// <param name="val">Object value.</param>
+        void WriteObject<T>(T val);
+
+        /// <summary>
+        /// Write object array.
+        /// </summary>
+        /// <param name="val">Object array.</param>
+        void WriteObjectArray<T>(T[] val);
+
+        /// <summary>
+        /// Write collection.
+        /// </summary>
+        /// <param name="val">Collection.</param>
+        void WriteCollection(ICollection val);
+
+        /// <summary>
+        /// Write generic collection.
+        /// </summary>
+        /// <param name="val">Collection.</param>
+        void WriteGenericCollection<T>(ICollection<T> val);
+
+        /// <summary>
+        /// Write dictionary.
+        /// </summary>
+        /// <param name="val">Dictionary.</param>
+        void WriteDictionary(IDictionary val);
+
+        /// <summary>
+        /// Write generic dictionary.
+        /// </summary>
+        /// <param name="val">Dictionary.</param>
+        void WriteGenericDictionary<TK, TV>(IDictionary<TK, TV> val);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableReader.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableReader.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableReader.cs
new file mode 100644
index 0000000..71bd4f2
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableReader.cs
@@ -0,0 +1,340 @@
+/*
+ * 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.Portable
+{
+    using System;
+    using System.Collections;
+    using System.Collections.Generic;
+
+    /// <summary>
+    /// Delegate for collection creation.
+    /// </summary>
+    /// <param name="size">Collection size.</param>
+    /// <returns>Collection.</returns>
+    public delegate ICollection PortableCollectionFactory(int size);
+
+    /// <summary>
+    /// Delegate for adding element to collection.
+    /// </summary>
+    /// <param name="col">Collection.</param>
+    /// <param name="elem">Element to add.</param>
+    public delegate void PortableCollectionAdder(ICollection col, object elem);
+
+    /// <summary>
+    /// Delegate for generic collection creation.
+    /// </summary>
+    /// <param name="size">Collection size.</param>
+    /// <returns>Collection.</returns>
+    public delegate ICollection<T> PortableGenericCollectionFactory<T>(int size);
+
+    /// <summary>
+    /// Delegate for dictionary creation.
+    /// </summary>
+    /// <param name="size">Dictionary size.</param>
+    /// <returns>Dictionary.</returns>
+    public delegate IDictionary PortableDictionaryFactory(int size);
+
+    /// <summary>
+    /// Delegate for generic collection creation.
+    /// </summary>
+    /// <param name="size">Collection size.</param>
+    /// <returns>Collection.</returns>
+    public delegate IDictionary<TK, TV> PortableGenericDictionaryFactory<TK, TV>(int size);
+
+    /// <summary>
+    /// Reader for portable objects. 
+    /// </summary>
+    public interface IPortableReader 
+    {
+        /// <summary>
+        /// Read named byte value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Byte value.</returns>
+        byte ReadByte(string fieldName);
+        
+        /// <summary>
+        /// Read named byte array. 
+        /// </summary>
+        /// <returns>Byte array.</returns>
+        byte[] ReadByteArray(string fieldName);
+        
+        /// <summary>
+        /// Read named char value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Char value.</returns>
+        char ReadChar(string fieldName);
+
+        /// <summary>
+        /// Read named char array. 
+        /// </summary>
+        /// <returns>Char array.</returns>
+        char[] ReadCharArray(string fieldName);
+
+        /// <summary>
+        /// Read named short value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Short value.</returns>
+        short ReadShort(string fieldName);
+
+        /// <summary>
+        /// Read named short array. 
+        /// </summary>
+        /// <returns>Short array.</returns>
+        short[] ReadShortArray(string fieldName);        
+
+        /// <summary>
+        /// Read named int value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Int value.</returns>
+        int ReadInt(string fieldName);
+
+        /// <summary>
+        /// Read named int array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Int array.</returns>
+        int[] ReadIntArray(string fieldName);
+
+        /// <summary>
+        /// Read named long value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Long value.</returns>
+        long ReadLong(string fieldName);
+
+        /// <summary>
+        /// Read named long array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Long array.</returns>
+        long[] ReadLongArray(string fieldName);
+
+        /// <summary>
+        /// Read named boolean value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Boolean value.</returns>
+        bool ReadBoolean(string fieldName);
+
+        /// <summary>
+        /// Read named boolean array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Boolean array.</returns>
+        bool[] ReadBooleanArray(string fieldName);
+
+        /// <summary>
+        /// Read named float value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Float value.</returns>
+        float ReadFloat(string fieldName);
+
+        /// <summary>
+        /// Read named float array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Float array.</returns>
+        float[] ReadFloatArray(string fieldName);
+
+        /// <summary>
+        /// Read named double value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Double value.</returns>
+        double ReadDouble(string fieldName);        
+
+        /// <summary>
+        /// Read named double array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Double array.</returns>
+        double[] ReadDoubleArray(string fieldName);
+
+        /// <summary>
+        /// Read named decimal value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Decimal value.</returns>
+        decimal ReadDecimal(string fieldName);
+
+        /// <summary>
+        /// Read named decimal array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Decimal array.</returns>
+        decimal[] ReadDecimalArray(string fieldName);
+
+        /// <summary>
+        /// Read named date value in UTC form. Shortcut for <c>ReadDate(fieldName, false)</c>.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Date value.</returns>
+        DateTime? ReadDate(string fieldName);
+
+        /// <summary>
+        /// Read named date value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="local">Whether to read date in local (<c>true</c>) or UTC (<c>false</c>) form.</param>
+        /// <returns>Date vaule.</returns>
+        DateTime? ReadDate(string fieldName, bool local);
+
+        /// <summary>
+        /// Read named date array in UTC form. Shortcut for <c>ReadDateArray(fieldName, false)</c>.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Date array.</returns>
+        DateTime?[] ReadDateArray(string fieldName);
+
+        /// <summary>
+        /// Read named date array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="local">Whether to read date in local (<c>true</c>) or UTC (<c>false</c>) form.</param>
+        /// <returns>Date array.</returns>
+        DateTime?[] ReadDateArray(string fieldName, bool local);
+
+        /// <summary>
+        /// Read named string value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>String value.</returns>
+        string ReadString(string fieldName);
+
+        /// <summary>
+        /// Read named string array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>String array.</returns>
+        string[] ReadStringArray(string fieldName);
+
+        /// <summary>
+        /// Read named GUID value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>GUID value.</returns>
+        Guid? ReadGuid(string fieldName);
+
+        /// <summary>
+        /// Read named GUID array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>GUID array.</returns>
+        Guid?[] ReadGuidArray(string fieldName);
+        
+        /// <summary>
+        /// Read named enum value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Enum value.</returns>
+        T ReadEnum<T>(string fieldName);
+
+        /// <summary>
+        /// Read named enum array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Enum array.</returns>
+        T[] ReadEnumArray<T>(string fieldName);
+
+        /// <summary>
+        /// Read named object.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Object.</returns>
+        T ReadObject<T>(string fieldName);
+
+        /// <summary>
+        /// Read named object array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Object array.</returns>
+        T[] ReadObjectArray<T>(string fieldName);
+
+        /// <summary>
+        /// Read named collection.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Collection.</returns>
+        ICollection ReadCollection(string fieldName);
+
+        /// <summary>
+        /// Read named collection.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="factory">Factory.</param>
+        /// <param name="adder">Adder.</param>
+        /// <returns>Collection.</returns>
+        ICollection ReadCollection(string fieldName, PortableCollectionFactory factory, PortableCollectionAdder adder);
+
+        /// <summary>
+        /// Read named generic collection.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Collection.</returns>
+        ICollection<T> ReadGenericCollection<T>(string fieldName);
+
+        /// <summary>
+        /// Read named generic collection.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="factory">Factory.</param>
+        /// <returns>Collection.</returns>
+        ICollection<T> ReadGenericCollection<T>(string fieldName, PortableGenericCollectionFactory<T> factory);
+
+        /// <summary>
+        /// Read named dictionary.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Dictionary.</returns>
+        IDictionary ReadDictionary(string fieldName);
+
+        /// <summary>
+        /// Read named dictionary.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="factory">Factory.</param>
+        /// <returns>Dictionary.</returns>
+        IDictionary ReadDictionary(string fieldName, PortableDictionaryFactory factory);
+
+        /// <summary>
+        /// Read named generic dictionary.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Dictionary.</returns>
+        IDictionary<TK, TV> ReadGenericDictionary<TK, TV>(string fieldName);
+
+        /// <summary>
+        /// Read named generic dictionary.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="factory">Factory.</param>
+        /// <returns>Dictionary.</returns>
+        IDictionary<TK, TV> ReadGenericDictionary<TK, TV>(string fieldName, PortableGenericDictionaryFactory<TK, TV> factory);
+
+        /// <summary>
+        /// Get raw reader. 
+        /// </summary>
+        /// <returns>Raw reader.</returns>
+        IPortableRawReader RawReader();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableSerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableSerializer.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableSerializer.cs
new file mode 100644
index 0000000..ac40dd7
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableSerializer.cs
@@ -0,0 +1,39 @@
+/*
+ * 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.Portable
+{
+    /// <summary>
+    /// Portable serializer. 
+    /// </summary> 
+    public interface IPortableSerializer
+    {
+        /// <summary>
+        /// Write portalbe object.
+        /// </summary>
+        /// <param name="obj">Object.</param>
+        /// <param name="writer">Poratble writer.</param>
+        void WritePortable(object obj, IPortableWriter writer);
+
+        /// <summary>
+        /// Read portable object.
+        /// </summary>
+        /// <param name="obj">Instantiated empty object.</param>
+        /// <param name="reader">Poratble reader.</param>
+        void ReadPortable(object obj, IPortableReader reader);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableWriter.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableWriter.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableWriter.cs
new file mode 100644
index 0000000..8df2d50
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableWriter.cs
@@ -0,0 +1,259 @@
+/*
+ * 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.Portable 
+{
+    using System;
+    using System.Collections;
+    using System.Collections.Generic;
+
+    /// <summary>
+    /// Writer for portable objects. 
+    /// </summary>
+    public interface IPortableWriter 
+    {
+        /// <summary>
+        /// Write named byte value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Byte value.</param>
+        void WriteByte(string fieldName, byte val);
+
+        /// <summary>
+        /// Write named byte array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Byte array.</param>
+        void WriteByteArray(string fieldName, byte[] val);
+
+        /// <summary>
+        /// Write named char value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Char value.</param>
+        void WriteChar(string fieldName, char val);
+
+        /// <summary>
+        /// Write named char array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Char array.</param>
+        void WriteCharArray(string fieldName, char[] val);
+
+        /// <summary>
+        /// Write named short value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Short value.</param>
+        void WriteShort(string fieldName, short val);
+
+        /// <summary>
+        /// Write named short array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Short array.</param>
+        void WriteShortArray(string fieldName, short[] val);
+
+        /// <summary>
+        /// Write named int value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Int value.</param>
+        void WriteInt(string fieldName, int val);
+
+        /// <summary>
+        /// Write named int array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Int array.</param>
+        void WriteIntArray(string fieldName, int[] val);
+
+        /// <summary>
+        /// Write named long value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Long value.</param>
+        void WriteLong(string fieldName, long val);
+
+        /// <summary>
+        /// Write named long array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Long array.</param>
+        void WriteLongArray(string fieldName, long[] val);
+
+        /// <summary>
+        /// Write named boolean value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Boolean value.</param>
+        void WriteBoolean(string fieldName, bool val);
+
+        /// <summary>
+        /// Write named boolean array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Boolean array.</param>
+        void WriteBooleanArray(string fieldName, bool[] val);
+
+        /// <summary>
+        /// Write named float value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Float value.</param>
+        void WriteFloat(string fieldName, float val);
+
+        /// <summary>
+        /// Write named float array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Float array.</param>
+        void WriteFloatArray(string fieldName, float[] val);
+
+        /// <summary>
+        /// Write named double value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Double value.</param>
+        void WriteDouble(string fieldName, double val);
+
+        /// <summary>
+        /// Write named double array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Double array.</param>
+        void WriteDoubleArray(string fieldName, double[] val);
+
+        /// <summary>
+        /// Write named decimal value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Decimal value.</param>
+        void WriteDecimal(string fieldName, decimal val);
+
+        /// <summary>
+        /// Write named decimal array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Decimal array.</param>
+        void WriteDecimalArray(string fieldName, decimal[] val);
+
+        /// <summary>
+        /// Write named date value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Date value.</param>
+        void WriteDate(string fieldName, DateTime? val);
+
+        /// <summary>
+        /// Write named date array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Date array.</param>
+        void WriteDateArray(string fieldName, DateTime?[] val);
+
+        /// <summary>
+        /// Write named string value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">String value.</param>
+        void WriteString(string fieldName, string val);
+
+        /// <summary>
+        /// Write named string array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">String array.</param>
+        void WriteStringArray(string fieldName, string[] val);
+
+        /// <summary>
+        /// Write named GUID value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">GUID value.</param>
+        void WriteGuid(string fieldName, Guid? val);
+
+        /// <summary>
+        /// Write named GUID array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">GUID array.</param>
+        void WriteGuidArray(string fieldName, Guid?[] val);
+
+        /// <summary>
+        /// Write named enum value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Enum value.</param>
+        void WriteEnum<T>(string fieldName, T val);
+
+        /// <summary>
+        /// Write named enum array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Enum array.</param>
+        void WriteEnumArray<T>(string fieldName, T[] val);
+
+        /// <summary>
+        /// Write named object value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Object value.</param>
+        void WriteObject<T>(string fieldName, T val);
+
+        /// <summary>
+        /// Write named object array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Object array.</param>
+        void WriteObjectArray<T>(string fieldName, T[] val);
+
+        /// <summary>
+        /// Write named collection.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Collection.</param>
+        void WriteCollection(string fieldName, ICollection val);
+
+        /// <summary>
+        /// Write named generic collection.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Collection.</param>
+        void WriteGenericCollection<T>(string fieldName, ICollection<T> val);
+
+        /// <summary>
+        /// Write named dictionary.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Dictionary.</param>
+        void WriteDictionary(string fieldName, IDictionary val);
+
+        /// <summary>
+        /// Write named generic dictionary.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Dictionary.</param>
+        void WriteGenericDictionary<TK, TV>(string fieldName, IDictionary<TK, TV> val);
+
+        /// <summary>
+        /// Get raw writer. 
+        /// </summary>
+        /// <returns>Raw writer.</returns>
+        IPortableRawWriter RawWriter();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortables.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortables.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortables.cs
new file mode 100644
index 0000000..905eda1
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortables.cs
@@ -0,0 +1,120 @@
+/*
+ * 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.Portable
+{
+    using System;
+    using System.Collections.Generic;
+
+    /// <summary>
+    /// Defines portable objects functionality. With portable objects you are able to:
+    /// <list type="bullet">
+    ///     <item>
+    ///         <description>Seamlessly interoperate between Java, .NET, and C++.</description>
+    ///     </item>
+    ///     <item>
+    ///         <description>Make any object portable with zero code change to your existing code.</description>
+    ///     </item>
+    ///     <item>
+    ///         <description>Nest portable objects within each other.</description>
+    ///     </item>
+    ///     <item>
+    ///         <description>Automatically handle <c>circular</c> or <c>null</c> references.</description>
+    ///     </item>
+    ///     <item>
+    ///         <description>Automatically convert collections and maps between Java, .NET, and C++.</description>
+    ///     </item>
+    ///     <item>
+    ///         <description>Optionally avoid deserialization of objects on the server side.</description>
+    ///     </item>
+    ///     <item>
+    ///         <description>Avoid need to have concrete class definitions on the server side.</description>
+    ///     </item>
+    ///     <item>
+    ///         <description>Dynamically change structure of the classes without having to restart the cluster.</description>
+    ///     </item>
+    ///     <item>
+    ///         <description>Index into portable objects for querying purposes.</description>
+    ///     </item>
+    /// </list>
+    /// </summary>
+    public interface IPortables
+    {
+        /// <summary>
+        /// Converts provided object to portable form.
+        /// <para />
+        /// Note that object's type needs to be configured in <see cref="PortableConfiguration"/>.
+        /// </summary>
+        /// <param name="obj">Object to convert.</param>
+        /// <returns>Converted object.</returns>
+        T ToPortable<T>(object obj);
+
+        /// <summary>
+        /// Create builder for the given portable object type. Note that this
+        /// type must be specified in <see cref="PortableConfiguration"/>.
+        /// </summary>
+        /// <param name="type"></param>
+        /// <returns>Builder.</returns>
+        IPortableBuilder Builder(Type type);
+
+        /// <summary>
+        /// Create builder for the given portable object type name. Note that this
+        /// type name must be specified in <see cref="PortableConfiguration"/>.
+        /// </summary>
+        /// <param name="typeName">Type name.</param>
+        /// <returns>Builder.</returns>
+        IPortableBuilder Builder(string typeName);
+
+        /// <summary>
+        /// Create builder over existing portable object.
+        /// </summary>
+        /// <param name="obj"></param>
+        /// <returns>Builder.</returns>
+        IPortableBuilder Builder(IPortableObject obj);
+
+        /// <summary>
+        /// Gets type id for the given type name.
+        /// </summary>
+        /// <param name="typeName">Type name.</param>
+        /// <returns>Type id.</returns>
+        int GetTypeId(string typeName);
+
+        /// <summary>
+        /// Gets metadata for all known types.
+        /// </summary>
+        /// <returns>Metadata.</returns>
+        ICollection<IPortableMetadata> GetMetadata();
+
+        /// <summary>
+        /// Gets metadata for specified type id.
+        /// </summary>
+        /// <returns>Metadata.</returns>
+        IPortableMetadata GetMetadata(int typeId);
+
+        /// <summary>
+        /// Gets metadata for specified type name.
+        /// </summary>
+        /// <returns>Metadata.</returns>
+        IPortableMetadata GetMetadata(string typeName);
+
+        /// <summary>
+        /// Gets metadata for specified type.
+        /// </summary>
+        /// <returns>Metadata.</returns>
+        IPortableMetadata GetMetadata(Type type);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/PortableConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/PortableConfiguration.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/PortableConfiguration.cs
new file mode 100644
index 0000000..39878c2
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/PortableConfiguration.cs
@@ -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.
+ */
+
+namespace Apache.Ignite.Core.Portable
+{
+    using System.Collections.Generic;
+
+    /// <summary>
+    /// Portable type configuration.
+    /// </summary>
+    public class PortableConfiguration
+    {
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        public PortableConfiguration()
+        {
+            DefaultMetadataEnabled = true;
+            DefaultKeepDeserialized = true;
+        }
+
+        /// <summary>
+        /// Copying constructor.
+        /// </summary>
+        /// <param name="cfg">Configuration to copy.</param>
+        public PortableConfiguration(PortableConfiguration cfg)
+        {
+            DefaultIdMapper = cfg.DefaultIdMapper;
+            DefaultNameMapper = cfg.DefaultNameMapper;
+            DefaultMetadataEnabled = cfg.DefaultMetadataEnabled;
+            DefaultKeepDeserialized = cfg.DefaultKeepDeserialized;
+            DefaultSerializer = cfg.DefaultSerializer;
+
+            Types = cfg.Types != null ? new List<string>(cfg.Types) : null;
+
+            if (cfg.TypeConfigurations != null)
+            {
+                TypeConfigurations = new List<PortableTypeConfiguration>(cfg.TypeConfigurations.Count);
+
+                foreach (PortableTypeConfiguration typeCfg in cfg.TypeConfigurations) 
+                    TypeConfigurations.Add(new PortableTypeConfiguration(typeCfg));
+            }
+        }
+
+        /// <summary>
+        /// Type configurations.
+        /// </summary>
+        public ICollection<PortableTypeConfiguration> TypeConfigurations
+        {
+            get;
+            set;
+        }
+
+        /// <summary>
+        /// Portable types. Shorthand for creating PortableTypeConfiguration.
+        /// </summary>
+        public ICollection<string> Types
+        {
+            get;
+            set;
+        }
+
+        /// <summary>
+        /// Default name mapper.
+        /// </summary>
+        public IPortableNameMapper DefaultNameMapper
+        {
+            get;
+            set;
+        }
+
+        /// <summary>
+        /// Default ID mapper.
+        /// </summary>
+        public IPortableIdMapper DefaultIdMapper
+        {
+            get;
+            set;
+        }
+
+        /// <summary>
+        /// Default serializer.
+        /// </summary>
+        public IPortableSerializer DefaultSerializer
+        {
+            get;
+            set;
+        }
+
+        /// <summary>
+        /// Default metadata enabled flag. Defaults to true.
+        /// </summary>
+        public bool DefaultMetadataEnabled
+        {
+            get;
+            set;
+        }
+
+        /// <summary>
+        /// Default keep deserialized flag.
+        /// </summary>
+        public bool DefaultKeepDeserialized
+        {
+            get;
+            set;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/PortableException.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/PortableException.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/PortableException.cs
new file mode 100644
index 0000000..95edbc0
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/PortableException.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.Portable 
+{
+    using System;
+    using System.Runtime.Serialization;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// Indicates an error during portable marshalling.
+    /// </summary>
+    [Serializable]
+    public class PortableException : IgniteException
+    {
+        /// <summary>
+        /// Constructs an exception. 
+        /// </summary>
+        public PortableException() 
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="PortableException"/> class.
+        /// </summary>
+        /// <param name="message">The message that describes the error.</param>
+        public PortableException(string message)
+            : base(message) {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="PortableException"/> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        public PortableException(string message, Exception cause)
+            : base(message, cause) {
+        }
+
+        /// <summary>
+        /// Constructs an exception.
+        /// </summary>
+        /// <param name="info">Serialization info.</param>
+        /// <param name="ctx">Streaming context.</param>
+        protected PortableException(SerializationInfo info, StreamingContext ctx)
+            : base(info, ctx) {
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/PortableTypeConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/PortableTypeConfiguration.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/PortableTypeConfiguration.cs
new file mode 100644
index 0000000..bbbd4a8
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/PortableTypeConfiguration.cs
@@ -0,0 +1,162 @@
+/*
+ * 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.Portable
+{
+    using System;
+
+    /// <summary>
+    /// Portable type configuration.
+    /// </summary>
+    public class PortableTypeConfiguration
+    {
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        public PortableTypeConfiguration()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="typeName">Type name.</param>
+        public PortableTypeConfiguration(string typeName)
+        {
+            TypeName = typeName;
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="type">Type.</param> 
+        public PortableTypeConfiguration(Type type)
+        {
+            TypeName = type.FullName;
+        }
+
+        /// <summary>
+        /// Copying constructor.
+        /// </summary>
+        /// <param name="cfg">Configuration to copy.</param>
+        public PortableTypeConfiguration(PortableTypeConfiguration cfg)
+        {
+            AffinityKeyFieldName = cfg.AffinityKeyFieldName;
+            AssemblyName = cfg.AssemblyName;
+            IdMapper = cfg.IdMapper;
+            NameMapper = cfg.NameMapper;
+            Serializer = cfg.Serializer;
+            TypeName = cfg.TypeName;
+            MetadataEnabled = cfg.MetadataEnabled;
+            KeepDeserialized = cfg.KeepDeserialized;
+        }
+
+        /// <summary>
+        /// Assembly name. 
+        /// </summary>
+        public string AssemblyName
+        {
+            get;
+            set;
+        }
+
+        /// <summary>
+        /// Fully qualified type name. 
+        /// </summary>
+        public string TypeName
+        {
+            get;
+            set;
+        }
+        
+        /// <summary>
+        /// Name mapper for the given type. 
+        /// </summary>
+        public IPortableNameMapper NameMapper
+        {
+            get;
+            set;
+        }
+
+        /// <summary>
+        /// ID mapper for the given type. When it is necessary to resolve class (field) ID, then 
+        /// this property will be checked first. If not set, then PortableClassIdAttribute 
+        /// (PortableFieldIdAttribute) will be checked in class through reflection. If required
+        /// attribute is not set, then ID will be hash code of the class (field) simple name in lower case. 
+        /// </summary>
+        public IPortableIdMapper IdMapper
+        {
+            get;
+            set;
+        }
+
+        /// <summary>
+        /// Serializer for the given type. If not provided and class implements IPortable
+        /// then its custom logic will be used. If not provided and class doesn't implement IPortable
+        /// then all fields of the class except of those with [NotSerialized] attribute will be serialized
+        ///with help of reflection.
+        /// </summary>
+        public IPortableSerializer Serializer
+        {
+            get;
+            set;
+        }
+
+        /// <summary>
+        /// Affinity key field name.
+        /// </summary>
+        public string AffinityKeyFieldName
+        {
+            get;
+            set;
+        }
+
+        /// <summary>
+        /// Metadata enabled flag. If set to non-null value, overrides default value set in 
+        /// PortableConfiguration.
+        /// </summary>
+        public bool? MetadataEnabled
+        {
+            get;
+            set;
+        }
+
+        /// <summary>
+        /// Keep deserialized flag. If set to non-null value, overrides default value set in 
+        /// PortableConfiguration.
+        /// </summary>
+        public bool? KeepDeserialized
+        {
+            get;
+            set;
+        }
+
+        /// <summary>
+        /// Returns a string that represents the current object.
+        /// </summary>
+        /// <returns>
+        /// A string that represents the current object.
+        /// </returns>
+        override public string ToString()
+        {
+            return typeof(PortableTypeConfiguration).Name + " [TypeName=" + TypeName + 
+                ", NameMapper=" +  NameMapper + ", IdMapper=" + IdMapper + ", Serializer=" + Serializer +
+                ", AffinityKeyFieldName=" + AffinityKeyFieldName + ']';
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/PortableTypeNames.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/PortableTypeNames.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/PortableTypeNames.cs
new file mode 100644
index 0000000..ed792c3
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/PortableTypeNames.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.Portable
+{
+    /// <summary>
+    /// Portable type name constants.
+    /// </summary>
+    public static class PortableTypeNames
+    {
+        /** Type name: boolean. */
+        public const string TypeNameBool = "boolean";
+
+        /** Type name: byte. */
+        public const string TypeNameByte = "byte";
+
+        /** Type name: short. */
+        public const string TypeNameShort = "short";
+
+        /** Type name: char. */
+        public const string TypeNameChar = "char";
+
+        /** Type name: int. */
+        public const string TypeNameInt = "int";
+
+        /** Type name: long. */
+        public const string TypeNameLong = "long";
+
+        /** Type name: float. */
+        public const string TypeNameFloat = "float";
+
+        /** Type name: double. */
+        public const string TypeNameDouble = "double";
+
+        /** Type name: decimal. */
+        public const string TypeNameDecimal = "decimal";
+
+        /** Type name: String. */
+        public const string TypeNameString = "String";
+
+        /** Type name: UUID. */
+        public const string TypeNameGuid = "UUID";
+
+        /** Type name: Date. */
+        public const string TypeNameDate = "Date";
+
+        /** Type name: Enum. */
+        public const string TypeNameEnum = "Enum";
+
+        /** Type name: Object. */
+        public const string TypeNameObject = "Object";
+
+        /** Type name: boolean array. */
+        public const string TypeNameArrayBool = "boolean[]";
+
+        /** Type name: byte array. */
+        public const string TypeNameArrayByte = "byte[]";
+
+        /** Type name: short array. */
+        public const string TypeNameArrayShort = "short[]";
+
+        /** Type name: char array. */
+        public const string TypeNameArrayChar = "char[]";
+
+        /** Type name: int array. */
+        public const string TypeNameArrayInt = "int[]";
+
+        /** Type name: long array. */
+        public const string TypeNameArrayLong = "long[]";
+
+        /** Type name: float array. */
+        public const string TypeNameArrayFloat = "float[]";
+
+        /** Type name: double array. */
+        public const string TypeNameArrayDouble = "double[]";
+
+        /** Type name: decimal array. */
+        public const string TypeNameArrayDecimal = "decimal[]";
+
+        /** Type name: String array. */
+        public const string TypeNameArrayString = "String[]";
+
+        /** Type name: UUID array. */
+        public const string TypeNameArrayGuid = "UUID[]";
+
+        /** Type name: Date array. */
+        public const string TypeNameArrayDate = "Date[]";
+
+        /** Type name: Enum array. */
+        public const string TypeNameArrayEnum = "Enum[]";
+
+        /** Type name: Object array. */
+        public const string TypeNameArrayObject = "Object[]";
+
+        /** Type name: Collection. */
+        public const string TypeNameCollection = "Collection";
+
+        /** Type name: Map. */
+        public const string TypeNameMap = "Map";
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Resource/InstanceResourceAttribute.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Resource/InstanceResourceAttribute.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Resource/InstanceResourceAttribute.cs
new file mode 100644
index 0000000..8b34c10
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Resource/InstanceResourceAttribute.cs
@@ -0,0 +1,35 @@
+/*
+ * 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.Resource
+{
+    using System;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Impl.Compute;
+
+    /// <summary>
+    /// Attribute which injects <see cref="IIgnite"/> instance. Can be defined inside
+    /// implementors of <see cref="IComputeTask{A,T,TR}"/> and <see cref="IComputeJob"/> interfaces.
+    /// Can be applied to non-static fields, properties and methods returning <c>void</c> and 
+    /// accepting a single parameter.
+    /// </summary>
+    [AttributeUsage(AttributeTargets.Field | AttributeTargets.Method | AttributeTargets.Property)]
+    public sealed class InstanceResourceAttribute : Attribute
+    {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Resource/StoreSessionResourceAttribute.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Resource/StoreSessionResourceAttribute.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Resource/StoreSessionResourceAttribute.cs
new file mode 100644
index 0000000..624c71d
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Resource/StoreSessionResourceAttribute.cs
@@ -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.
+ */
+
+namespace Apache.Ignite.Core.Resource
+{
+    using System;
+    using Apache.Ignite.Core.Cache.Store;
+
+    /// <summary>
+    /// Annotates a field or a setter method for injection of current <see cref="ICacheStoreSession"/>
+    /// instance. It can be injected into <see cref="ICacheStore"/>.
+    /// </summary>
+    [AttributeUsage(AttributeTargets.Field | AttributeTargets.Method | AttributeTargets.Property)]
+    public sealed class StoreSessionResourceAttribute : Attribute
+    {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Services/IService.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Services/IService.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Services/IService.cs
new file mode 100644
index 0000000..3668221
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Services/IService.cs
@@ -0,0 +1,51 @@
+/*
+ * 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.Services
+{
+    /// <summary>
+    /// Represents Ignite-managed service.
+    /// </summary>
+    public interface IService
+    {
+        /// <summary>
+        /// Initializes this instance before execution.
+        /// </summary>
+        /// <param name="context">Service execution context.</param>
+        void Init(IServiceContext context);
+
+        /// <summary>
+        /// Starts execution of this service. This method is automatically invoked whenever an instance of the service
+        /// is deployed on a Ignite node. Note that service is considered deployed even after it exits the Execute
+        /// method and can be cancelled (or undeployed) only by calling any of the Cancel methods on 
+        /// <see cref="IServices"/> API. Also note that service is not required to exit from Execute method until
+        /// Cancel method was called.
+        /// </summary>
+        /// <param name="context">Service execution context.</param>
+        void Execute(IServiceContext context);
+
+        /// <summary>
+        /// Cancels this instance.
+        /// <para/>
+        /// Note that Ignite cannot guarantee that the service exits from <see cref="IService.Execute"/>
+        /// method whenever <see cref="IService.Cancel"/> is called. It is up to the user to
+        /// make sure that the service code properly reacts to cancellations.
+        /// </summary>
+        /// <param name="context">Service execution context.</param>
+        void Cancel(IServiceContext context);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Services/IServiceContext.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Services/IServiceContext.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Services/IServiceContext.cs
new file mode 100644
index 0000000..50c3f14
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Services/IServiceContext.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.Services
+{
+    using System;
+
+    /// <summary>
+    /// Represents service execution context.
+    /// </summary>
+    public interface IServiceContext
+    {
+        /// <summary>
+        /// Gets service name.
+        /// </summary>
+        /// <returns>
+        /// Service name.
+        /// </returns>
+        string Name { get; }
+
+        /// <summary>
+        /// Gets service execution ID. Execution ID is guaranteed to be unique across all service deployments.
+        /// </summary>
+        /// <returns>
+        /// Service execution ID.
+        /// </returns>
+        Guid ExecutionId { get; }
+
+        /// <summary>
+        /// Get flag indicating whether service has been cancelled or not.
+        /// </summary>
+        /// <returns>
+        /// Flag indicating whether service has been cancelled or not.
+        /// </returns>
+        bool IsCancelled { get; }
+
+        /// <summary>
+        /// Gets cache name used for key-to-node affinity calculation. 
+        /// This parameter is optional and is set only when key-affinity service was deployed.
+        /// </summary>
+        /// <returns>
+        /// Cache name, possibly null.
+        /// </returns>
+        string CacheName { get; }
+
+        /// <summary>
+        /// Gets affinity key used for key-to-node affinity calculation. 
+        /// This parameter is optional and is set only when key-affinity service was deployed.
+        /// </summary>
+        /// <value>
+        /// Affinity key, possibly null.
+        /// </value>
+        object AffinityKey { get; }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Services/IServiceDescriptor.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Services/IServiceDescriptor.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Services/IServiceDescriptor.cs
new file mode 100644
index 0000000..96bad4f
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Services/IServiceDescriptor.cs
@@ -0,0 +1,96 @@
+/*
+ * 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.Services
+{
+    using System;
+    using System.Collections.Generic;
+
+    /// <summary>
+    /// Service deployment descriptor.
+    /// </summary>
+    public interface IServiceDescriptor
+    {
+        /// <summary>
+        /// Gets service name.
+        /// </summary>
+        /// <returns>
+        /// Service name.
+        /// </returns>
+        string Name { get; }
+
+        /// <summary>
+        /// Gets the service type.
+        /// </summary>
+        /// <value>
+        /// Service type.
+        /// </value>
+        Type Type { get; }
+
+        /// <summary>
+        /// Gets maximum allowed total number of deployed services in the grid, 0 for unlimited.
+        /// </summary>
+        /// <returns>
+        /// Maximum allowed total number of deployed services in the grid, 0 for unlimited.
+        /// </returns>
+        int TotalCount { get; }
+
+        /// <summary>
+        /// Gets maximum allowed number of deployed services on each node, 0 for unlimited.
+        /// </summary>
+        /// <returns>
+        /// Maximum allowed total number of deployed services on each node, 0 for unlimited.
+        /// </returns>
+        int MaxPerNodeCount { get; }
+
+        /// <summary>
+        /// Gets cache name used for key-to-node affinity calculation. 
+        /// This parameter is optional and is set only when key-affinity service was deployed.
+        /// </summary>
+        /// <returns>
+        /// Cache name, possibly null.
+        /// </returns>
+        string CacheName { get; }
+
+        /// <summary>
+        /// Gets affinity key used for key-to-node affinity calculation. 
+        /// This parameter is optional and is set only when key-affinity service was deployed.
+        /// </summary>
+        /// <value>
+        /// Affinity key, possibly null.
+        /// </value>
+        object AffinityKey { get; }
+
+        /// <summary>
+        /// Gets affinity key used for key-to-node affinity calculation. 
+        /// This parameter is optional and is set only when key-affinity service was deployed.
+        /// </summary>
+        /// <returns>
+        /// Affinity key, possibly null.
+        /// </returns>
+        Guid OriginNodeId { get; }
+
+        /// <summary>
+        /// Gets service deployment topology snapshot. Service topology snapshot is represented
+        /// by number of service instances deployed on a node mapped to node ID.
+        /// </summary>
+        /// <value>
+        /// Map of number of service instances per node ID.
+        /// </value>
+        IDictionary<Guid, int> TopologySnapshot { get; }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Services/IServices.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Services/IServices.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Services/IServices.cs
new file mode 100644
index 0000000..fff25c3
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Services/IServices.cs
@@ -0,0 +1,181 @@
+/*
+ * 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.Services
+{
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// Defines functionality to deploy distributed services in the Ignite.
+    /// </summary>
+    public interface IServices : IAsyncSupport<IServices>
+    {
+        /// <summary>
+        /// Gets the cluster group to which this instance belongs.
+        /// </summary>
+        /// <value>
+        /// The cluster group to which this instance belongs.
+        /// </value>
+        IClusterGroup ClusterGroup { get; }
+
+        /// <summary>
+        /// Deploys a cluster-wide singleton service. Ignite guarantees that there is always
+        /// one instance of the service in the cluster. In case if Ignite node on which the service
+        /// was deployed crashes or stops, Ignite will automatically redeploy it on another node.
+        /// However, if the node on which the service is deployed remains in topology, then the
+        /// service will always be deployed on that node only, regardless of topology changes.
+        /// <para />
+        /// Note that in case of topology changes, due to network delays, there may be a temporary situation
+        /// when a singleton service instance will be active on more than one node (e.g. crash detection delay).
+        /// </summary>
+        /// <param name="name">Service name.</param>
+        /// <param name="service">Service instance.</param>
+        [AsyncSupported]
+        void DeployClusterSingleton(string name, IService service);
+
+        /// <summary>
+        /// Deploys a per-node singleton service. Ignite guarantees that there is always
+        /// one instance of the service running on each node. Whenever new nodes are started
+        /// within the underlying cluster group, Ignite will automatically deploy one instance of
+        /// the service on every new node.        
+        /// </summary>
+        /// <param name="name">Service name.</param>
+        /// <param name="service">Service instance.</param>
+        [AsyncSupported]
+        void DeployNodeSingleton(string name, IService service);
+
+        /// <summary>
+        /// Deploys one instance of this service on the primary node for a given affinity key.
+        /// Whenever topology changes and primary node assignment changes, Ignite will always
+        /// make sure that the service is undeployed on the previous primary node and deployed
+        /// on the new primary node.
+        /// <para />
+        /// Note that in case of topology changes, due to network delays, there may be a temporary situation
+        /// when a service instance will be active on more than one node (e.g. crash detection delay).
+        /// </summary>
+        /// <param name="name">Service name.</param>
+        /// <param name="service">Service instance.</param>
+        /// <param name="cacheName">Name of the cache on which affinity for key should be calculated, null for
+        /// default cache.</param>
+        /// <param name="affinityKey">Affinity cache key.</param>
+        [AsyncSupported]
+        void DeployKeyAffinitySingleton<TK>(string name, IService service, string cacheName, TK affinityKey);
+
+        /// <summary>
+        /// Deploys multiple instances of the service on the grid. Ignite will deploy a
+        /// maximum amount of services equal to <paramref name="totalCount" /> parameter making sure that
+        /// there are no more than <paramref name="maxPerNodeCount" /> service instances running
+        /// on each node. Whenever topology changes, Ignite will automatically rebalance
+        /// the deployed services within cluster to make sure that each node will end up with
+        /// about equal number of deployed instances whenever possible.
+        /// </summary>
+        /// <param name="name">Service name.</param>
+        /// <param name="service">Service instance.</param>
+        /// <param name="totalCount">Maximum number of deployed services in the grid, 0 for unlimited.</param>
+        /// <param name="maxPerNodeCount">Maximum number of deployed services on each node, 0 for unlimited.</param>
+        [AsyncSupported]
+        void DeployMultiple(string name, IService service, int totalCount, int maxPerNodeCount);
+
+        /// <summary>
+        /// Deploys instances of the service in the Ignite according to provided configuration.
+        /// </summary>
+        /// <param name="configuration">Service configuration.</param>
+        [AsyncSupported]
+        void Deploy(ServiceConfiguration configuration);
+
+        /// <summary>
+        /// Cancels service deployment. If a service with specified name was deployed on the grid, 
+        /// then <see cref="IService.Cancel"/> method will be called on it.
+        /// <para/>
+        /// Note that Ignite cannot guarantee that the service exits from <see cref="IService.Execute"/>
+        /// method whenever <see cref="IService.Cancel"/> is called. It is up to the user to
+        /// make sure that the service code properly reacts to cancellations.
+        /// </summary>
+        /// <param name="name">Name of the service to cancel.</param>
+        [AsyncSupported]
+        void Cancel(string name);
+
+        /// <summary>
+        /// Cancels all deployed services.
+        /// <para/>
+        /// Note that depending on user logic, it may still take extra time for a service to 
+        /// finish execution, even after it was cancelled.
+        /// </summary>
+        [AsyncSupported]
+        void CancelAll();
+
+        /// <summary>
+        /// Gets metadata about all deployed services.
+        /// </summary>
+        /// <returns>Metadata about all deployed services.</returns>
+        ICollection<IServiceDescriptor> GetServiceDescriptors();
+
+        /// <summary>
+        /// Gets deployed service with specified name.
+        /// </summary>
+        /// <typeparam name="T">Service type.</typeparam>
+        /// <param name="name">Service name.</param>
+        /// <returns>Deployed service with specified name.</returns>
+        T GetService<T>(string name);
+
+        /// <summary>
+        /// Gets all deployed services with specified name.
+        /// </summary>
+        /// <typeparam name="T">Service type.</typeparam>
+        /// <param name="name">Service name.</param>
+        /// <returns>All deployed services with specified name.</returns>
+        ICollection<T> GetServices<T>(string name);
+
+        /// <summary>
+        /// Gets a remote handle on the service. If service is available locally,
+        /// then local instance is returned, otherwise, a remote proxy is dynamically
+        /// created and provided for the specified service.
+        /// </summary>
+        /// <typeparam name="T">Service type.</typeparam>
+        /// <param name="name">Service name.</param>
+        /// <returns>Either proxy over remote service or local service if it is deployed locally.</returns>
+        T GetServiceProxy<T>(string name) where T : class;
+
+        /// <summary>
+        /// Gets a remote handle on the service. If service is available locally,
+        /// then local instance is returned, otherwise, a remote proxy is dynamically
+        /// created and provided for the specified service.
+        /// </summary>
+        /// <typeparam name="T">Service type.</typeparam>
+        /// <param name="name">Service name.</param>
+        /// <param name="sticky">Whether or not Ignite should always contact the same remote
+        /// service or try to load-balance between services.</param>
+        /// <returns>Either proxy over remote service or local service if it is deployed locally.</returns>
+        T GetServiceProxy<T>(string name, bool sticky) where T : class;
+
+        /// <summary>
+        /// Returns an instance with portable mode enabled.
+        /// Service method results will be kept in portable form.
+        /// </summary>
+        /// <returns>Instance with portable mode enabled.</returns>
+        IServices WithKeepPortable();
+        
+        /// <summary>
+        /// Returns an instance with server-side portable mode enabled.
+        /// Service method arguments will be kept in portable form.
+        /// </summary>
+        /// <returns>Instance with server-side portable mode enabled.</returns>
+        IServices WithServerKeepPortable();
+    }
+}
\ No newline at end of file


[26/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs
index 094c6a5..f151763 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs
@@ -15,13 +15,12 @@
  * limitations under the License.
  */
 
-using Apache.Ignite.Core.Impl.Portable.IO;
-
 namespace Apache.Ignite.Core.Impl.Memory
 {
     using System;
     using System.IO;
     using System.Text;
+    using Apache.Ignite.Core.Impl.Portable.IO;
 
     /// <summary>
     /// Platform memory stream.

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryUtils.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryUtils.cs
index fc942a0..2b0277a 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryUtils.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryUtils.cs
@@ -21,7 +21,7 @@ namespace Apache.Ignite.Core.Impl.Memory
     using System.Diagnostics.CodeAnalysis;
     using System.Reflection;
     using System.Runtime.InteropServices;
-    
+
     /// <summary>
     /// Utility methods for platform memory management.
     /// </summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformPooledMemory.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformPooledMemory.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformPooledMemory.cs
index 7709ca4..206df4b 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformPooledMemory.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformPooledMemory.cs
@@ -35,7 +35,7 @@ namespace Apache.Ignite.Core.Impl.Memory
         /// <param name="memPtr">Memory pointer.</param>
         public PlatformPooledMemory(PlatformMemoryPool pool, long memPtr) : base(memPtr)
         {
-            this._pool = pool;
+            _pool = pool;
         }
 
         /** <inheritdoc /> */

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Messaging/MessageFilterHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Messaging/MessageFilterHolder.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Messaging/MessageFilterHolder.cs
new file mode 100644
index 0000000..21c66bf
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Messaging/MessageFilterHolder.cs
@@ -0,0 +1,179 @@
+/*
+ * 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.Messaging
+{
+    using System;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Handle;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Impl.Resource;
+    using Apache.Ignite.Core.Messaging;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Non-generic portable filter wrapper.
+    /// </summary>
+    internal class MessageFilterHolder : IPortableWriteAware, IHandle
+    {
+        /** Invoker function that takes key and value and invokes wrapped IMessageFilter */
+        private readonly Func<Guid, object, bool> _invoker;
+
+        /** Current Ignite instance. */
+        private readonly Ignite _ignite;
+        
+        /** Underlying filter. */
+        private readonly object _filter;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="MessageFilterHolder" /> class.
+        /// </summary>
+        /// <param name="grid">Grid.</param>
+        /// <param name="filter">The <see cref="IMessageFilter{T}" /> to wrap.</param>
+        /// <param name="invoker">The invoker func that takes key and value and invokes wrapped IMessageFilter.</param>
+        private MessageFilterHolder(Ignite grid, object filter, Func<Guid, object, bool> invoker)
+        {
+            Debug.Assert(filter != null);
+            Debug.Assert(invoker != null);
+
+            _invoker = invoker;
+
+            _filter = filter;
+
+            // 1. Set fields.
+            Debug.Assert(grid != null);
+
+            _ignite = grid;
+            _invoker = invoker;
+
+            // 2. Perform injections.
+            ResourceProcessor.Inject(filter, grid);
+        }
+
+        /// <summary>
+        /// Invoke the filter.
+        /// </summary>
+        /// <param name="input">Input.</param>
+        /// <returns></returns>
+        public int Invoke(IPortableStream input)
+        {
+            var rawReader = _ignite.Marshaller.StartUnmarshal(input).RawReader();
+
+            var nodeId = rawReader.ReadGuid();
+
+            Debug.Assert(nodeId != null);
+
+            return _invoker(nodeId.Value, rawReader.ReadObject<object>()) ? 1 : 0;
+        }
+
+        /// <summary>
+        /// Wrapped <see cref="IMessageFilter{T}" />.
+        /// </summary>
+        public object Filter
+        {
+            get { return _filter; }
+        }
+
+        /// <summary>
+        /// Destroy callback.
+        /// </summary>
+        public Action DestroyAction { private get; set; }
+
+        /** <inheritDoc /> */
+        public void Release()
+        {
+            if (DestroyAction != null)
+                DestroyAction();
+        }
+
+        /** <inheritDoc /> */
+        public bool Released
+        {
+            get { return false; } // Multiple releases are allowed.
+        }
+
+        /// <summary>
+        /// Creates local holder instance.
+        /// </summary>
+        /// <param name="grid">Ignite instance.</param>
+        /// <param name="filter">Filter.</param>
+        /// <returns>
+        /// New instance of <see cref="MessageFilterHolder" />
+        /// </returns>
+        public static MessageFilterHolder CreateLocal<T>(Ignite grid, IMessageFilter<T> filter)
+        {
+            Debug.Assert(filter != null);
+
+            return new MessageFilterHolder(grid, filter, (id, msg) => filter.Invoke(id, (T)msg));
+        }
+
+        /// <summary>
+        /// Creates remote holder instance.
+        /// </summary>
+        /// <param name="grid">Grid.</param>
+        /// <param name="memPtr">Memory pointer.</param>
+        /// <returns>Deserialized instance of <see cref="MessageFilterHolder"/></returns>
+        public static MessageFilterHolder CreateRemote(Ignite grid, long memPtr)
+        {
+            Debug.Assert(grid != null);
+            
+            var stream = IgniteManager.Memory.Get(memPtr).Stream();
+
+            var holder = grid.Marshaller.Unmarshal<MessageFilterHolder>(stream);
+
+            return holder;
+        }
+
+        /// <summary>
+        /// Gets the invoker func.
+        /// </summary>
+        private static Func<Guid, object, bool> GetInvoker(object pred)
+        {
+            var func = DelegateTypeDescriptor.GetMessageFilter(pred.GetType());
+
+            return (id, msg) => func(pred, id, msg);
+        }
+
+        /** <inheritdoc /> */
+        public void WritePortable(IPortableWriter writer)
+        {
+            var writer0 = (PortableWriterImpl)writer.RawWriter();
+
+            writer0.DetachNext();
+            PortableUtils.WritePortableOrSerializable(writer0, Filter);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="MessageFilterHolder"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        public MessageFilterHolder(IPortableReader reader)
+        {
+            var reader0 = (PortableReaderImpl)reader.RawReader();
+
+            _filter = PortableUtils.ReadPortableOrSerializable<object>(reader0);
+
+            _invoker = GetInvoker(_filter);
+
+            _ignite = reader0.Marshaller.Ignite;
+
+            ResourceProcessor.Inject(_filter, _ignite);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Messaging/Messaging.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Messaging/Messaging.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Messaging/Messaging.cs
new file mode 100644
index 0000000..e8c4b4b
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Messaging/Messaging.cs
@@ -0,0 +1,262 @@
+/*
+ * 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.Messaging
+{
+    using System;
+    using System.Collections;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using System.Linq;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Collections;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Resource;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+    using Apache.Ignite.Core.Messaging;
+    using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils;
+
+    /// <summary>
+    /// Messaging functionality.
+    /// </summary>
+    internal class Messaging : PlatformTarget, IMessaging
+    {
+        /// <summary>
+        /// Opcodes.
+        /// </summary>
+        private enum Op
+        {
+            LocalListen = 1,
+            RemoteListen = 2,
+            Send = 3,
+            SendMulti = 4,
+            SendOrdered = 5,
+            StopLocalListen = 6,
+            StopRemoteListen = 7
+        }
+
+        /** Map from user (func+topic) -> id, needed for unsubscription. */
+        private readonly MultiValueDictionary<KeyValuePair<object, object>, long> _funcMap =
+            new MultiValueDictionary<KeyValuePair<object, object>, long>();
+
+        /** Grid */
+        private readonly Ignite _ignite;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="Messaging" /> class.
+        /// </summary>
+        /// <param name="target">Target.</param>
+        /// <param name="marsh">Marshaller.</param>
+        /// <param name="prj">Cluster group.</param>
+        public Messaging(IUnmanagedTarget target, PortableMarshaller marsh, IClusterGroup prj)
+            : base(target, marsh)
+        {
+            Debug.Assert(prj != null);
+
+            ClusterGroup = prj;
+
+            _ignite = (Ignite) prj.Ignite;
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ClusterGroup { get; private set; }
+
+        /** <inheritdoc /> */
+        public void Send(object message, object topic = null)
+        {
+            IgniteArgumentCheck.NotNull(message, "message");
+
+            DoOutOp((int) Op.Send, topic, message);
+        }
+
+        /** <inheritdoc /> */
+        public void Send(IEnumerable messages, object topic = null)
+        {
+            IgniteArgumentCheck.NotNull(messages, "messages");
+
+            DoOutOp((int) Op.SendMulti, writer =>
+            {
+                writer.Write(topic);
+
+                WriteEnumerable(writer, messages.OfType<object>());
+            });
+        }
+
+        /** <inheritdoc /> */
+        public void SendOrdered(object message, object topic = null, TimeSpan? timeout = null)
+        {
+            IgniteArgumentCheck.NotNull(message, "message");
+
+            DoOutOp((int) Op.SendOrdered, writer =>
+            {
+                writer.Write(topic);
+                writer.Write(message);
+
+                writer.WriteLong((long)(timeout == null ? 0 : timeout.Value.TotalMilliseconds));
+            });
+        }
+
+        /** <inheritdoc /> */
+        public void LocalListen<T>(IMessageFilter<T> filter, object topic = null)
+        {
+            IgniteArgumentCheck.NotNull(filter, "filter");
+
+            ResourceProcessor.Inject(filter, _ignite);
+
+            lock (_funcMap)
+            {
+                var key = GetKey(filter, topic);
+
+                MessageFilterHolder filter0 = MessageFilterHolder.CreateLocal(_ignite, filter); 
+
+                var filterHnd = _ignite.HandleRegistry.Allocate(filter0);
+
+                filter0.DestroyAction = () =>
+                {
+                    lock (_funcMap)
+                    {
+                        _funcMap.Remove(key, filterHnd);
+                    }
+                };
+
+                try
+                {
+                    DoOutOp((int) Op.LocalListen, writer =>
+                    {
+                        writer.WriteLong(filterHnd);
+                        writer.Write(topic);
+                    });
+                }
+                catch (Exception)
+                {
+                    _ignite.HandleRegistry.Release(filterHnd);
+
+                    throw;
+                }
+
+                _funcMap.Add(key, filterHnd);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public void StopLocalListen<T>(IMessageFilter<T> filter, object topic = null)
+        {
+            IgniteArgumentCheck.NotNull(filter, "filter");
+
+            long filterHnd;
+            bool removed;
+
+            lock (_funcMap)
+            {
+                removed = _funcMap.TryRemove(GetKey(filter, topic), out filterHnd);
+            }
+
+            if (removed)
+            {
+                DoOutOp((int) Op.StopLocalListen, writer =>
+                {
+                    writer.WriteLong(filterHnd);
+                    writer.Write(topic);
+                });
+            }
+        }
+
+        /** <inheritdoc /> */
+        public Guid RemoteListen<T>(IMessageFilter<T> filter, object topic = null)
+        {
+            IgniteArgumentCheck.NotNull(filter, "filter");
+
+            var filter0 = MessageFilterHolder.CreateLocal(_ignite, filter);
+            var filterHnd = _ignite.HandleRegistry.AllocateSafe(filter0);
+
+            try
+            {
+                Guid id = Guid.Empty;
+
+                DoOutInOp((int) Op.RemoteListen, writer =>
+                {
+                    writer.Write(filter0);
+                    writer.WriteLong(filterHnd);
+                    writer.Write(topic);
+                }, 
+                input =>
+                {
+                    var id0 = Marshaller.StartUnmarshal(input).RawReader().ReadGuid();
+
+                    Debug.Assert(IsAsync || id0.HasValue);
+
+                    if (id0.HasValue)
+                        id = id0.Value;
+                });
+
+                return id;
+            }
+            catch (Exception)
+            {
+                _ignite.HandleRegistry.Release(filterHnd);
+
+                throw;
+            }
+        }
+
+        /** <inheritdoc /> */
+        public void StopRemoteListen(Guid opId)
+        {
+            DoOutOp((int) Op.StopRemoteListen, writer =>
+            {
+                writer.WriteGuid(opId);
+            });
+        }
+
+        /** <inheritdoc /> */
+        public virtual IMessaging WithAsync()
+        {
+            return new MessagingAsync(UU.MessagingWithASync(Target), Marshaller, ClusterGroup);
+        }
+
+        /** <inheritdoc /> */
+        public virtual bool IsAsync
+        {
+            get { return false; }
+        }
+
+        /** <inheritdoc /> */
+        public virtual IFuture GetFuture()
+        {
+            throw IgniteUtils.GetAsyncModeDisabledException();
+        }
+
+        /** <inheritdoc /> */
+        public virtual IFuture<TResult> GetFuture<TResult>()
+        {
+            throw IgniteUtils.GetAsyncModeDisabledException();
+        }
+
+        /// <summary>
+        /// Gets the key for user-provided filter and topic.
+        /// </summary>
+        /// <param name="filter">Filter.</param>
+        /// <param name="topic">Topic.</param>
+        /// <returns>Compound dictionary key.</returns>
+        private static KeyValuePair<object, object> GetKey(object filter, object topic)
+        {
+            return new KeyValuePair<object, object>(filter, topic);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Messaging/MessagingAsync.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Messaging/MessagingAsync.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Messaging/MessagingAsync.cs
new file mode 100644
index 0000000..e899d4e
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Messaging/MessagingAsync.cs
@@ -0,0 +1,68 @@
+/*
+ * 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.Messaging
+{
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+    using Apache.Ignite.Core.Messaging;
+    using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils;
+
+    /// <summary>
+    /// Async messaging implementation.
+    /// </summary>
+    internal class MessagingAsync : Messaging
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="MessagingAsync" /> class.
+        /// </summary>
+        /// <param name="target">Target.</param>
+        /// <param name="marsh">Marshaller.</param>
+        /// <param name="prj">Cluster group.</param>
+        public MessagingAsync(IUnmanagedTarget target, PortableMarshaller marsh, 
+            IClusterGroup prj) : base(target, marsh, prj)
+        {
+            // No-op.
+        }
+
+        /** <inheritdoc /> */
+        public override IMessaging WithAsync()
+        {
+            return this;
+        }
+
+        /** <inheritdoc /> */
+        public override bool IsAsync
+        {
+            get { return true; }
+        }
+
+        /** <inheritdoc /> */
+        public override IFuture GetFuture()
+        {
+            return GetFuture<object>();
+        }
+
+        /** <inheritdoc /> */
+        public override IFuture<T> GetFuture<T>()
+        {
+            return GetFuture<T>((futId, futTyp) => UU.TargetListenFuture(Target, futId, futTyp));
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/NativeMethods.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/NativeMethods.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/NativeMethods.cs
new file mode 100644
index 0000000..6e25e7e
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/NativeMethods.cs
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Impl
+{
+    using System;
+    using System.Runtime.InteropServices;
+
+    /// <summary>
+    /// Native methods.
+    /// </summary>
+    internal static class NativeMethods
+    {
+        /// <summary>
+        /// Load DLL with WinAPI.
+        /// </summary>
+        /// <param name="path">Path to dll.</param>
+        /// <returns></returns>
+        [DllImport("kernel32.dll", SetLastError = true, CharSet = CharSet.Ansi, BestFitMapping = false, 
+            ThrowOnUnmappableChar = true)]
+        internal static extern IntPtr LoadLibrary(string path);
+
+        /// <summary>
+        /// Get procedure address with WinAPI.
+        /// </summary>
+        /// <param name="ptr">DLL pointer.</param>
+        /// <param name="name">Procedure name.</param>
+        /// <returns>Procedure address.</returns>
+        [DllImport("kernel32.dll", SetLastError = true, CharSet = CharSet.Ansi, BestFitMapping = false, 
+            ThrowOnUnmappableChar = true)]
+        internal static extern IntPtr GetProcAddress(IntPtr ptr, string name);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
new file mode 100644
index 0000000..67f631a
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
@@ -0,0 +1,715 @@
+/*
+ * 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
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Diagnostics.CodeAnalysis;
+    using System.IO;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Memory;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Impl.Portable.Metadata;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+    using Apache.Ignite.Core.Portable;
+    using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils;
+
+    /// <summary>
+    /// Base class for interop targets.
+    /// </summary>
+    [SuppressMessage("ReSharper", "LocalVariableHidesMember")]
+    internal abstract class PlatformTarget
+    {
+        /** */
+        protected const int True = 1;
+
+        /** */
+        private const int OpMeta = -1;
+
+        /** */
+        public const int OpNone = -2;
+
+        /** */
+        private static readonly Dictionary<Type, FutureType> IgniteFutureTypeMap
+            = new Dictionary<Type, FutureType>
+            {
+                {typeof(bool), FutureType.Bool},
+                {typeof(byte), FutureType.Byte},
+                {typeof(char), FutureType.Char},
+                {typeof(double), FutureType.Double},
+                {typeof(float), FutureType.Float},
+                {typeof(int), FutureType.Int},
+                {typeof(long), FutureType.Long},
+                {typeof(short), FutureType.Short}
+            };
+        
+        /** Unmanaged target. */
+        private readonly IUnmanagedTarget _target;
+
+        /** Marshaller. */
+        private readonly PortableMarshaller _marsh;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="target">Target.</param>
+        /// <param name="marsh">Marshaller.</param>
+        protected PlatformTarget(IUnmanagedTarget target, PortableMarshaller marsh)
+        {
+            _target = target;
+            _marsh = marsh;
+        }
+
+        /// <summary>
+        /// Unmanaged target.
+        /// </summary>
+        internal IUnmanagedTarget Target
+        {
+            get { return _target; }
+        }
+
+        /// <summary>
+        /// Marshaller.
+        /// </summary>
+        internal PortableMarshaller Marshaller
+        {
+            get { return _marsh; }
+        }
+
+        #region Static Helpers
+
+        /// <summary>
+        /// Write collection.
+        /// </summary>
+        /// <param name="writer">Portable writer.</param>
+        /// <param name="vals">Values.</param>
+        /// <returns>The same writer for chaining.</returns>
+        protected static PortableWriterImpl WriteCollection<T>(PortableWriterImpl writer, ICollection<T> vals)
+        {
+            return WriteCollection<T, T>(writer, vals, null);
+        }
+
+        /// <summary>
+        /// Write nullable collection.
+        /// </summary>
+        /// <param name="writer">Portable writer.</param>
+        /// <param name="vals">Values.</param>
+        /// <returns>The same writer for chaining.</returns>
+        protected static PortableWriterImpl WriteNullableCollection<T>(PortableWriterImpl writer, ICollection<T> vals)
+        {
+            return WriteNullable(writer, vals, WriteCollection);
+        }
+
+        /// <summary>
+        /// Write collection.
+        /// </summary>
+        /// <param name="writer">Portable writer.</param>
+        /// <param name="vals">Values.</param>
+        /// <param name="selector">A transform function to apply to each element.</param>
+        /// <returns>The same writer for chaining.</returns>
+        protected static PortableWriterImpl WriteCollection<T1, T2>(PortableWriterImpl writer,
+            ICollection<T1> vals, Func<T1, T2> selector)
+        {
+            writer.WriteInt(vals.Count);
+
+            if (selector == null)
+            {
+                foreach (var val in vals)
+                    writer.Write(val);
+            }
+            else
+            {
+                foreach (var val in vals)
+                    writer.Write(selector(val));
+            }
+
+            return writer;
+        }
+
+        /// <summary>
+        /// Write enumerable.
+        /// </summary>
+        /// <param name="writer">Portable writer.</param>
+        /// <param name="vals">Values.</param>
+        /// <returns>The same writer for chaining.</returns>
+        protected static PortableWriterImpl WriteEnumerable<T>(PortableWriterImpl writer, IEnumerable<T> vals)
+        {
+            return WriteEnumerable<T, T>(writer, vals, null);
+        }
+
+        /// <summary>
+        /// Write enumerable.
+        /// </summary>
+        /// <param name="writer">Portable writer.</param>
+        /// <param name="vals">Values.</param>
+        /// <param name="selector">A transform function to apply to each element.</param>
+        /// <returns>The same writer for chaining.</returns>
+        protected static PortableWriterImpl WriteEnumerable<T1, T2>(PortableWriterImpl writer,
+            IEnumerable<T1> vals, Func<T1, T2> selector)
+        {
+            var col = vals as ICollection<T1>;
+
+            if (col != null)
+                return WriteCollection(writer, col, selector);
+            
+            var stream = writer.Stream;
+
+            var pos = stream.Position;
+
+            stream.Seek(4, SeekOrigin.Current);
+
+            var size = 0;
+
+            if (selector == null)
+            {
+                foreach (var val in vals)
+                {
+                    writer.Write(val);
+
+                    size++;
+                }
+            }
+            else
+            {
+                foreach (var val in vals)
+                {
+                    writer.Write(selector(val));
+
+                    size++;
+                }
+            }
+
+            stream.WriteInt(pos, size);
+                
+            return writer;
+        }
+
+        /// <summary>
+        /// Write dictionary.
+        /// </summary>
+        /// <param name="writer">Portable writer.</param>
+        /// <param name="vals">Values.</param>
+        /// <returns>The same writer.</returns>
+        protected static PortableWriterImpl WriteDictionary<T1, T2>(PortableWriterImpl writer, 
+            IDictionary<T1, T2> vals)
+        {
+            writer.WriteInt(vals.Count);
+
+            foreach (KeyValuePair<T1, T2> pair in vals)
+            {
+                writer.Write(pair.Key);
+                writer.Write(pair.Value);
+            }
+
+            return writer;
+        }
+
+        /// <summary>
+        /// Write a nullable item.
+        /// </summary>
+        /// <param name="writer">Portable writer.</param>
+        /// <param name="item">Item.</param>
+        /// <param name="writeItem">Write action to perform on item when it is not null.</param>
+        /// <returns>The same writer for chaining.</returns>
+        protected static PortableWriterImpl WriteNullable<T>(PortableWriterImpl writer, T item,
+            Func<PortableWriterImpl, T, PortableWriterImpl> writeItem)
+        {
+            if (item == null)
+            {
+                writer.WriteBoolean(false);
+
+                return writer;
+            }
+
+            writer.WriteBoolean(true);
+
+            return writeItem(writer, item);
+        }
+
+        #endregion
+
+        #region OUT operations
+
+        /// <summary>
+        /// Perform out operation.
+        /// </summary>
+        /// <param name="type">Operation type.</param>
+        /// <param name="action">Action to be performed on the stream.</param>
+        /// <returns></returns>
+        protected long DoOutOp(int type, Action<IPortableStream> action)
+        {
+            using (var stream = IgniteManager.Memory.Allocate().Stream())
+            {
+                action(stream);
+
+                return UU.TargetInStreamOutLong(_target, type, stream.SynchronizeOutput());
+            }
+        }
+
+        /// <summary>
+        /// Perform out operation.
+        /// </summary>
+        /// <param name="type">Operation type.</param>
+        /// <param name="action">Action to be performed on the stream.</param>
+        /// <returns></returns>
+        protected long DoOutOp(int type, Action<PortableWriterImpl> action)
+        {
+            using (var stream = IgniteManager.Memory.Allocate().Stream())
+            {
+                var writer = _marsh.StartMarshal(stream);
+
+                action(writer);
+
+                FinishMarshal(writer);
+
+                return UU.TargetInStreamOutLong(_target, type, stream.SynchronizeOutput());
+            }
+        }
+
+        /// <summary>
+        /// Perform simple output operation accepting single argument.
+        /// </summary>
+        /// <param name="type">Operation type.</param>
+        /// <param name="val1">Value.</param>
+        /// <returns>Result.</returns>
+        protected long DoOutOp<T1>(int type, T1 val1)
+        {
+            return DoOutOp(type, writer =>
+            {
+                writer.Write(val1);
+            });
+        }
+
+        /// <summary>
+        /// Perform simple output operation accepting two arguments.
+        /// </summary>
+        /// <param name="type">Operation type.</param>
+        /// <param name="val1">Value 1.</param>
+        /// <param name="val2">Value 2.</param>
+        /// <returns>Result.</returns>
+        protected long DoOutOp<T1, T2>(int type, T1 val1, T2 val2)
+        {
+            return DoOutOp(type, writer =>
+            {
+                writer.Write(val1);
+                writer.Write(val2);
+            });
+        }
+
+        /// <summary>
+        /// Perform simple output operation accepting three arguments.
+        /// </summary>
+        /// <param name="type">Operation type.</param>
+        /// <param name="val1">Value 1.</param>
+        /// <param name="val2">Value 2.</param>
+        /// <param name="val3">Value 3.</param>
+        /// <returns>Result.</returns>
+        protected long DoOutOp<T1, T2, T3>(int type, T1 val1, T2 val2, T3 val3)
+        {
+            return DoOutOp(type, writer =>
+            {
+                writer.Write(val1);
+                writer.Write(val2);
+                writer.Write(val3);
+            });
+        }
+
+        #endregion
+
+        #region IN operations
+
+        /// <summary>
+        /// Perform in operation.
+        /// </summary>
+        /// <param name="type">Type.</param>
+        /// <param name="action">Action.</param>
+        protected void DoInOp(int type, Action<IPortableStream> action)
+        {
+            using (var stream = IgniteManager.Memory.Allocate().Stream())
+            {
+                UU.TargetOutStream(_target, type, stream.MemoryPointer);
+                
+                stream.SynchronizeInput();
+
+                action(stream);
+            }
+        }
+
+        /// <summary>
+        /// Perform in operation.
+        /// </summary>
+        /// <param name="type">Type.</param>
+        /// <param name="action">Action.</param>
+        /// <returns>Result.</returns>
+        protected T DoInOp<T>(int type, Func<IPortableStream, T> action)
+        {
+            using (var stream = IgniteManager.Memory.Allocate().Stream())
+            {
+                UU.TargetOutStream(_target, type, stream.MemoryPointer);
+
+                stream.SynchronizeInput();
+
+                return action(stream);
+            }
+        }
+
+        /// <summary>
+        /// Perform simple in operation returning immediate result.
+        /// </summary>
+        /// <param name="type">Type.</param>
+        /// <returns>Result.</returns>
+        protected T DoInOp<T>(int type)
+        {
+            using (var stream = IgniteManager.Memory.Allocate().Stream())
+            {
+                UU.TargetOutStream(_target, type, stream.MemoryPointer);
+
+                stream.SynchronizeInput();
+
+                return Unmarshal<T>(stream);
+            }
+        }
+
+        #endregion
+
+        #region OUT-IN operations
+        
+        /// <summary>
+        /// Perform out-in operation.
+        /// </summary>
+        /// <param name="type">Operation type.</param>
+        /// <param name="outAction">Out action.</param>
+        /// <param name="inAction">In action.</param>
+        protected void DoOutInOp(int type, Action<PortableWriterImpl> outAction, Action<IPortableStream> inAction)
+        {
+            using (PlatformMemoryStream outStream = IgniteManager.Memory.Allocate().Stream())
+            {
+                using (PlatformMemoryStream inStream = IgniteManager.Memory.Allocate().Stream())
+                {
+                    PortableWriterImpl writer = _marsh.StartMarshal(outStream);
+
+                    outAction(writer);
+
+                    FinishMarshal(writer);
+
+                    UU.TargetInStreamOutStream(_target, type, outStream.SynchronizeOutput(), inStream.MemoryPointer);
+
+                    inStream.SynchronizeInput();
+
+                    inAction(inStream);
+                }
+            }
+        }
+
+        /// <summary>
+        /// Perform out-in operation.
+        /// </summary>
+        /// <param name="type">Operation type.</param>
+        /// <param name="outAction">Out action.</param>
+        /// <param name="inAction">In action.</param>
+        /// <returns>Result.</returns>
+        protected TR DoOutInOp<TR>(int type, Action<PortableWriterImpl> outAction, Func<IPortableStream, TR> inAction)
+        {
+            using (PlatformMemoryStream outStream = IgniteManager.Memory.Allocate().Stream())
+            {
+                using (PlatformMemoryStream inStream = IgniteManager.Memory.Allocate().Stream())
+                {
+                    PortableWriterImpl writer = _marsh.StartMarshal(outStream);
+
+                    outAction(writer);
+
+                    FinishMarshal(writer);
+
+                    UU.TargetInStreamOutStream(_target, type, outStream.SynchronizeOutput(), inStream.MemoryPointer);
+
+                    inStream.SynchronizeInput();
+
+                    return inAction(inStream);
+                }
+            }
+        }
+        
+        /// <summary>
+        /// Perform out-in operation.
+        /// </summary>
+        /// <param name="type">Operation type.</param>
+        /// <param name="outAction">Out action.</param>
+        /// <param name="inAction">In action.</param>
+        /// <param name="arg">Argument.</param>
+        /// <returns>Result.</returns>
+        protected unsafe TR DoOutInOp<TR>(int type, Action<PortableWriterImpl> outAction, Func<IPortableStream, TR> inAction, void* arg)
+        {
+            using (PlatformMemoryStream outStream = IgniteManager.Memory.Allocate().Stream())
+            {
+                using (PlatformMemoryStream inStream = IgniteManager.Memory.Allocate().Stream())
+                {
+                    PortableWriterImpl writer = _marsh.StartMarshal(outStream);
+
+                    outAction(writer);
+
+                    FinishMarshal(writer);
+
+                    UU.TargetInObjectStreamOutStream(_target, type, arg, outStream.SynchronizeOutput(), inStream.MemoryPointer);
+
+                    inStream.SynchronizeInput();
+
+                    return inAction(inStream);
+                }
+            }
+        }
+        
+        /// <summary>
+        /// Perform out-in operation.
+        /// </summary>
+        /// <param name="type">Operation type.</param>
+        /// <param name="outAction">Out action.</param>
+        /// <returns>Result.</returns>
+        protected TR DoOutInOp<TR>(int type, Action<PortableWriterImpl> outAction)
+        {
+            using (PlatformMemoryStream outStream = IgniteManager.Memory.Allocate().Stream())
+            {
+                using (PlatformMemoryStream inStream = IgniteManager.Memory.Allocate().Stream())
+                {
+                    PortableWriterImpl writer = _marsh.StartMarshal(outStream);
+
+                    outAction(writer);
+
+                    FinishMarshal(writer);
+
+                    UU.TargetInStreamOutStream(_target, type, outStream.SynchronizeOutput(), inStream.MemoryPointer);
+
+                    inStream.SynchronizeInput();
+
+                    return Unmarshal<TR>(inStream);
+                }
+            }
+        }
+
+        /// <summary>
+        /// Perform simple out-in operation accepting single argument.
+        /// </summary>
+        /// <param name="type">Operation type.</param>
+        /// <param name="val">Value.</param>
+        /// <returns>Result.</returns>
+        protected TR DoOutInOp<T1, TR>(int type, T1 val)
+        {
+            using (PlatformMemoryStream outStream = IgniteManager.Memory.Allocate().Stream())
+            {
+                using (PlatformMemoryStream inStream = IgniteManager.Memory.Allocate().Stream())
+                {
+                    PortableWriterImpl writer = _marsh.StartMarshal(outStream);
+
+                    writer.WriteObject(val);
+
+                    FinishMarshal(writer);
+
+                    UU.TargetInStreamOutStream(_target, type, outStream.SynchronizeOutput(), inStream.MemoryPointer);
+
+                    inStream.SynchronizeInput();
+
+                    return Unmarshal<TR>(inStream);
+                }
+            }
+        }
+
+        /// <summary>
+        /// Perform simple out-in operation accepting two arguments.
+        /// </summary>
+        /// <param name="type">Operation type.</param>
+        /// <param name="val1">Value.</param>
+        /// <param name="val2">Value.</param>
+        /// <returns>Result.</returns>
+        protected TR DoOutInOp<T1, T2, TR>(int type, T1 val1, T2 val2)
+        {
+            using (PlatformMemoryStream outStream = IgniteManager.Memory.Allocate().Stream())
+            {
+                using (PlatformMemoryStream inStream = IgniteManager.Memory.Allocate().Stream())
+                {
+                    PortableWriterImpl writer = _marsh.StartMarshal(outStream);
+
+                    writer.WriteObject(val1);
+                    writer.WriteObject(val2);
+
+                    FinishMarshal(writer);
+
+                    UU.TargetInStreamOutStream(_target, type, outStream.SynchronizeOutput(), inStream.MemoryPointer);
+
+                    inStream.SynchronizeInput();
+
+                    return Unmarshal<TR>(inStream);
+                }
+            }
+        }
+
+        #endregion
+
+        #region Miscelanneous
+
+        /// <summary>
+        /// Finish marshaling.
+        /// </summary>
+        /// <param name="writer">Portable writer.</param>
+        internal void FinishMarshal(PortableWriterImpl writer)
+        {
+            _marsh.FinishMarshal(writer);
+        }
+
+        /// <summary>
+        /// Put metadata to Grid.
+        /// </summary>
+        /// <param name="metas">Metadatas.</param>
+        internal void PutMetadata(IDictionary<int, IPortableMetadata> metas)
+        {
+            DoOutOp(OpMeta, stream =>
+            {
+                PortableWriterImpl metaWriter = _marsh.StartMarshal(stream);
+
+                metaWriter.WriteInt(metas.Count);
+
+                foreach (var meta in metas.Values)
+                {
+                    PortableMetadataImpl meta0 = (PortableMetadataImpl)meta;
+
+                    metaWriter.WriteInt(meta0.TypeId);
+                    metaWriter.WriteString(meta0.TypeName);
+                    metaWriter.WriteString(meta0.AffinityKeyFieldName);
+
+                    IDictionary<string, int> fields = meta0.FieldsMap();
+
+                    metaWriter.WriteInt(fields.Count);
+
+                    foreach (var field in fields)
+                    {
+                        metaWriter.WriteString(field.Key);
+                        metaWriter.WriteInt(field.Value);
+                    }
+                }
+
+                _marsh.FinishMarshal(metaWriter);
+            });
+
+            _marsh.OnMetadataSent(metas);
+        }
+
+        /// <summary>
+        /// Unmarshal object using the given stream.
+        /// </summary>
+        /// <param name="stream">Stream.</param>
+        /// <returns>Unmarshalled object.</returns>
+        protected virtual T Unmarshal<T>(IPortableStream stream)
+        {
+            return _marsh.Unmarshal<T>(stream);
+        }
+
+        /// <summary>
+        /// Creates a future and starts listening.
+        /// </summary>
+        /// <typeparam name="T">Future result type</typeparam>
+        /// <param name="listenAction">The listen action.</param>
+        /// <param name="keepPortable">Keep portable flag, only applicable to object futures. False by default.</param>
+        /// <param name="convertFunc">The function to read future result from stream.</param>
+        /// <returns>Created future.</returns>
+        protected IFuture<T> GetFuture<T>(Action<long, int> listenAction, bool keepPortable = false,
+            Func<PortableReaderImpl, T> convertFunc = null)
+        {
+            var futType = FutureType.Object;
+
+            var type = typeof(T);
+
+            if (type.IsPrimitive)
+                IgniteFutureTypeMap.TryGetValue(type, out futType);
+
+            var fut = convertFunc == null && futType != FutureType.Object
+                ? new Future<T>()
+                : new Future<T>(new FutureConverter<T>(_marsh, keepPortable, convertFunc));
+
+            var futHnd = _marsh.Ignite.HandleRegistry.Allocate(fut);
+
+            listenAction(futHnd, (int)futType);
+
+            return fut;
+        }
+
+        #endregion
+    }
+
+    /// <summary>
+    /// PlatformTarget with IDisposable pattern.
+    /// </summary>
+    internal abstract class PlatformDisposableTarget : PlatformTarget, IDisposable
+    {
+        /** Disposed flag. */
+        private volatile bool _disposed;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="target">Target.</param>
+        /// <param name="marsh">Marshaller.</param>
+        protected PlatformDisposableTarget(IUnmanagedTarget target, PortableMarshaller marsh) : base(target, marsh)
+        {
+            // No-op.
+        }
+
+        /** <inheritdoc /> */
+        public void Dispose()
+        {
+            lock (this)
+            {
+                if (_disposed)
+                    return;
+
+                Dispose(true);
+
+                GC.SuppressFinalize(this);
+
+                _disposed = true;
+            }
+        }
+
+        /// <summary>
+        /// Releases unmanaged and - optionally - managed resources.
+        /// </summary>
+        /// <param name="disposing">
+        /// <c>true</c> when called from Dispose;  <c>false</c> when called from finalizer.
+        /// </param>
+        protected virtual void Dispose(bool disposing)
+        {
+            Target.Dispose();
+        }
+
+        /// <summary>
+        /// Throws <see cref="ObjectDisposedException"/> if this instance has been disposed.
+        /// </summary>
+        protected void ThrowIfDisposed()
+        {
+            if (_disposed)
+                throw new ObjectDisposedException(GetType().Name, "Object has been disposed.");
+        }
+
+        /// <summary>
+        /// Gets a value indicating whether this instance is disposed.
+        /// </summary>
+        protected bool IsDisposed
+        {
+            get { return _disposed; }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/IPortableSystemTypeSerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/IPortableSystemTypeSerializer.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/IPortableSystemTypeSerializer.cs
new file mode 100644
index 0000000..3fee3ca
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/IPortableSystemTypeSerializer.cs
@@ -0,0 +1,34 @@
+/*
+ * 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.Portable
+{
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Serializer for system types that can create instances directly from a stream and does not support handles.
+    /// </summary>
+    internal interface IPortableSystemTypeSerializer : IPortableSerializer
+    {
+        /// <summary>
+        /// Reads the instance from a reader.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        /// <returns>Deserialized instance.</returns>
+        object ReadInstance(PortableReaderImpl reader);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/IPortableTypeDescriptor.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/IPortableTypeDescriptor.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/IPortableTypeDescriptor.cs
new file mode 100644
index 0000000..4a4f0dc
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/IPortableTypeDescriptor.cs
@@ -0,0 +1,124 @@
+/*
+ * 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.Portable
+{
+    using System;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Type descriptor.
+    /// </summary>
+    internal interface IPortableTypeDescriptor
+    {
+        /// <summary>
+        /// Type.
+        /// </summary>
+        Type Type
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Type ID.
+        /// </summary>
+        int TypeId
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Type name.
+        /// </summary>
+        string TypeName
+        {
+            get;
+        }
+
+        /// <summary>
+        /// User type flag.
+        /// </summary>
+        bool UserType
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Metadata enabled flag.
+        /// </summary>
+        bool MetadataEnabled
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Whether to cache deserialized value in IPortableObject
+        /// </summary>
+        bool KeepDeserialized
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Name converter.
+        /// </summary>
+        IPortableNameMapper NameConverter
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Mapper.
+        /// </summary>
+        IPortableIdMapper Mapper
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Serializer.
+        /// </summary>
+        IPortableSerializer Serializer
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Affinity key field name.
+        /// </summary>
+        string AffinityKeyFieldName
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Typed handler.
+        /// </summary>
+        object TypedHandler
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Untyped handler.
+        /// </summary>
+        PortableSystemWriteDelegate UntypedHandler
+        {
+            get;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/IPortableWriteAware.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/IPortableWriteAware.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/IPortableWriteAware.cs
new file mode 100644
index 0000000..d3c1521
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/IPortableWriteAware.cs
@@ -0,0 +1,34 @@
+/*
+ * 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.Portable
+{
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Represents an object that can write itself to a portable writer.
+    /// </summary>
+    internal interface IPortableWriteAware
+    {
+        /// <summary>
+        /// Writes this object to the given writer.
+        /// </summary> 
+        /// <param name="writer">Writer.</param>
+        /// <exception cref="System.IO.IOException">If write failed.</exception>
+        void WritePortable(IPortableWriter writer);
+    }
+}
\ No newline at end of file


[12/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryAbstractTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryAbstractTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryAbstractTest.cs
new file mode 100644
index 0000000..55bc76c
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryAbstractTest.cs
@@ -0,0 +1,1181 @@
+/*
+ * 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.Cache.Query.Continuous
+{
+    using System;
+    using System.Collections.Concurrent;
+    using System.Collections.Generic;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Linq;
+    using System.Runtime.Serialization;
+    using System.Threading;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cache.Event;
+    using Apache.Ignite.Core.Cache.Query;
+    using Apache.Ignite.Core.Cache.Query.Continuous;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.Portable;
+    using Apache.Ignite.Core.Resource;
+    using NUnit.Framework;
+    using CQU = Apache.Ignite.Core.Impl.Cache.Query.Continuous.ContinuousQueryUtils;
+
+    /// <summary>
+    /// Tests for continuous query.
+    /// </summary>
+    [SuppressMessage("ReSharper", "InconsistentNaming")]
+    [SuppressMessage("ReSharper", "PossibleNullReferenceException")]
+    [SuppressMessage("ReSharper", "StaticMemberInGenericType")]
+    public abstract class ContinuousQueryAbstractTest
+    {
+        /** Cache name: ATOMIC, backup. */
+        protected const string CACHE_ATOMIC_BACKUP = "atomic_backup";
+
+        /** Cache name: ATOMIC, no backup. */
+        protected const string CACHE_ATOMIC_NO_BACKUP = "atomic_no_backup";
+
+        /** Cache name: TRANSACTIONAL, backup. */
+        protected const string CACHE_TX_BACKUP = "transactional_backup";
+
+        /** Cache name: TRANSACTIONAL, no backup. */
+        protected const string CACHE_TX_NO_BACKUP = "transactional_no_backup";
+
+        /** Listener events. */
+        public static BlockingCollection<CallbackEvent> CB_EVTS = new BlockingCollection<CallbackEvent>();
+
+        /** Listener events. */
+        public static BlockingCollection<FilterEvent> FILTER_EVTS = new BlockingCollection<FilterEvent>();
+
+        /** First node. */
+        private IIgnite grid1;
+
+        /** Second node. */
+        private IIgnite grid2;
+
+        /** Cache on the first node. */
+        private ICache<int, PortableEntry> cache1;
+
+        /** Cache on the second node. */
+        private ICache<int, PortableEntry> cache2;
+
+        /** Cache name. */
+        private readonly string cacheName;
+        
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="cacheName">Cache name.</param>
+        protected ContinuousQueryAbstractTest(string cacheName)
+        {
+            this.cacheName = cacheName;
+        }
+
+        /// <summary>
+        /// Set-up routine.
+        /// </summary>
+        [TestFixtureSetUp]
+        public void SetUp()
+        {
+            GC.Collect();
+            TestUtils.JvmDebug = true;
+
+            IgniteConfigurationEx cfg = new IgniteConfigurationEx();
+
+            PortableConfiguration portCfg = new PortableConfiguration();
+
+            ICollection<PortableTypeConfiguration> portTypeCfgs = new List<PortableTypeConfiguration>();
+
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(PortableEntry)));
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(PortableFilter)));
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(KeepPortableFilter)));
+
+            portCfg.TypeConfigurations = portTypeCfgs;
+
+            cfg.PortableConfiguration = portCfg;
+            cfg.JvmClasspath = TestUtils.CreateTestClasspath();
+            cfg.JvmOptions = TestUtils.TestJavaOptions();
+            cfg.SpringConfigUrl = "config\\cache-query-continuous.xml";
+
+            cfg.GridName = "grid-1";
+            grid1 = Ignition.Start(cfg);
+            cache1 = grid1.Cache<int, PortableEntry>(cacheName);
+
+            cfg.GridName = "grid-2";
+            grid2 = Ignition.Start(cfg);
+            cache2 = grid2.Cache<int, PortableEntry>(cacheName);
+        }
+
+        /// <summary>
+        /// Tear-down routine.
+        /// </summary>
+        [TestFixtureTearDown]
+        public void TearDown()
+        {
+            Ignition.StopAll(true);
+        }
+
+        /// <summary>
+        /// Before-test routine.
+        /// </summary>
+        [SetUp]
+        public void BeforeTest()
+        {
+            CB_EVTS = new BlockingCollection<CallbackEvent>();
+            FILTER_EVTS = new BlockingCollection<FilterEvent>();
+
+            AbstractFilter<PortableEntry>.res = true;
+            AbstractFilter<PortableEntry>.err = false;
+            AbstractFilter<PortableEntry>.marshErr = false;
+            AbstractFilter<PortableEntry>.unmarshErr = false;
+
+            cache1.Remove(PrimaryKey(cache1));
+            cache1.Remove(PrimaryKey(cache2));
+
+            Assert.AreEqual(0, cache1.Size());
+            Assert.AreEqual(0, cache2.Size());
+
+            Console.WriteLine("Test started: " + TestContext.CurrentContext.Test.Name);
+        }
+        
+        /// <summary>
+        /// Test arguments validation.
+        /// </summary>
+        [Test]
+        public void TestValidation()
+        {
+            Assert.Throws<ArgumentException>(() => { cache1.QueryContinuous(new ContinuousQuery<int, PortableEntry>(null)); });
+        }
+
+        /// <summary>
+        /// Test multiple closes.
+        /// </summary>
+        [Test]
+        public void TestMultipleClose()
+        {
+            int key1 = PrimaryKey(cache1);
+            int key2 = PrimaryKey(cache2);
+
+            ContinuousQuery<int, PortableEntry> qry =
+                new ContinuousQuery<int, PortableEntry>(new Listener<PortableEntry>());
+
+            IDisposable qryHnd;
+
+            using (qryHnd = cache1.QueryContinuous(qry))
+            {
+                // Put from local node.
+                cache1.GetAndPut(key1, Entry(key1));
+                CheckCallbackSingle(key1, null, Entry(key1));
+
+                // Put from remote node.
+                cache2.GetAndPut(key2, Entry(key2));
+                CheckCallbackSingle(key2, null, Entry(key2));
+            }
+
+            qryHnd.Dispose();
+        }
+
+        /// <summary>
+        /// Test regular callback operations.
+        /// </summary>
+        [Test]
+        public void TestCallback()
+        {
+            CheckCallback(false);
+        }
+
+        /// <summary>
+        /// Check regular callback execution.
+        /// </summary>
+        /// <param name="loc"></param>
+        protected void CheckCallback(bool loc)
+        {
+            int key1 = PrimaryKey(cache1);
+            int key2 = PrimaryKey(cache2);
+            
+            ContinuousQuery<int, PortableEntry> qry = loc ?
+                new ContinuousQuery<int, PortableEntry>(new Listener<PortableEntry>(), true) :
+                new ContinuousQuery<int, PortableEntry>(new Listener<PortableEntry>());
+
+            using (cache1.QueryContinuous(qry))
+            {
+                // Put from local node.
+                cache1.GetAndPut(key1, Entry(key1));
+                CheckCallbackSingle(key1, null, Entry(key1));
+
+                cache1.GetAndPut(key1, Entry(key1 + 1));
+                CheckCallbackSingle(key1, Entry(key1), Entry(key1 + 1));
+
+                cache1.Remove(key1);
+                CheckCallbackSingle(key1, Entry(key1 + 1), null);
+
+                // Put from remote node.
+                cache2.GetAndPut(key2, Entry(key2));
+
+                if (loc)
+                    CheckNoCallback(100);
+                else
+                    CheckCallbackSingle(key2, null, Entry(key2));
+
+                cache1.GetAndPut(key2, Entry(key2 + 1));
+
+                if (loc)
+                    CheckNoCallback(100);
+                else
+                    CheckCallbackSingle(key2, Entry(key2), Entry(key2 + 1));
+
+                cache1.Remove(key2);
+
+                if (loc)
+                    CheckNoCallback(100);
+                else
+                    CheckCallbackSingle(key2, Entry(key2 + 1), null);
+            }
+
+            cache1.Put(key1, Entry(key1));
+            CheckNoCallback(100);
+
+            cache1.Put(key2, Entry(key2));
+            CheckNoCallback(100);
+        } 
+        
+        /// <summary>
+        /// Test Ignite injection into callback.
+        /// </summary>
+        [Test]
+        public void TestCallbackInjection()
+        {
+            Listener<PortableEntry> cb = new Listener<PortableEntry>();
+
+            Assert.IsNull(cb.ignite);
+
+            using (cache1.QueryContinuous(new ContinuousQuery<int, PortableEntry>(cb)))
+            {
+                Assert.IsNotNull(cb.ignite);
+            }
+        }
+        
+        /// <summary>
+        /// Test portable filter logic.
+        /// </summary>
+        [Test]
+        public void TestFilterPortable()
+        {
+            CheckFilter(true, false);
+        }
+
+        /// <summary>
+        /// Test serializable filter logic.
+        /// </summary>
+        [Test]
+        public void TestFilterSerializable()
+        {
+            CheckFilter(false, false);
+        }
+
+        /// <summary>
+        /// Check filter.
+        /// </summary>
+        /// <param name="portable">Portable.</param>
+        /// <param name="loc">Local cache flag.</param>
+        protected void CheckFilter(bool portable, bool loc)
+        {
+            ICacheEntryEventListener<int, PortableEntry> lsnr = new Listener<PortableEntry>();
+            ICacheEntryEventFilter<int, PortableEntry> filter = 
+                portable ? (AbstractFilter<PortableEntry>)new PortableFilter() : new SerializableFilter();
+
+            ContinuousQuery<int, PortableEntry> qry = loc ? 
+                new ContinuousQuery<int, PortableEntry>(lsnr, filter, true) : 
+                new ContinuousQuery<int, PortableEntry>(lsnr, filter);
+
+            using (cache1.QueryContinuous(qry))
+            {
+                // Put from local node.
+                int key1 = PrimaryKey(cache1);
+                cache1.GetAndPut(key1, Entry(key1));
+                CheckFilterSingle(key1, null, Entry(key1));
+                CheckCallbackSingle(key1, null, Entry(key1));
+
+                // Put from remote node.
+                int key2 = PrimaryKey(cache2);
+                cache1.GetAndPut(key2, Entry(key2));
+
+                if (loc)
+                {
+                    CheckNoFilter(key2);
+                    CheckNoCallback(key2);
+                }
+                else
+                {
+                    CheckFilterSingle(key2, null, Entry(key2));
+                    CheckCallbackSingle(key2, null, Entry(key2));
+                }
+
+                AbstractFilter<PortableEntry>.res = false;
+
+                // Ignored put from local node.
+                cache1.GetAndPut(key1, Entry(key1 + 1));
+                CheckFilterSingle(key1, Entry(key1), Entry(key1 + 1));
+                CheckNoCallback(100);
+
+                // Ignored put from remote node.
+                cache1.GetAndPut(key2, Entry(key2 + 1));
+
+                if (loc)
+                    CheckNoFilter(100);
+                else
+                    CheckFilterSingle(key2, Entry(key2), Entry(key2 + 1));
+
+                CheckNoCallback(100);
+            }
+        }
+
+        /// <summary>
+        /// Test portable filter error during invoke.
+        /// </summary>
+        [Ignore("IGNITE-521")]
+        [Test]
+        public void TestFilterInvokeErrorPortable()
+        {
+            CheckFilterInvokeError(true);
+        }
+
+        /// <summary>
+        /// Test serializable filter error during invoke.
+        /// </summary>
+        [Ignore("IGNITE-521")]
+        [Test]
+        public void TestFilterInvokeErrorSerializable()
+        {
+            CheckFilterInvokeError(false);
+        }
+
+        /// <summary>
+        /// Check filter error handling logic during invoke.
+        /// </summary>
+        private void CheckFilterInvokeError(bool portable)
+        {
+            AbstractFilter<PortableEntry>.err = true;
+
+            ICacheEntryEventListener<int, PortableEntry> lsnr = new Listener<PortableEntry>();
+            ICacheEntryEventFilter<int, PortableEntry> filter =
+                portable ? (AbstractFilter<PortableEntry>) new PortableFilter() : new SerializableFilter();
+
+            ContinuousQuery<int, PortableEntry> qry = new ContinuousQuery<int, PortableEntry>(lsnr, filter);
+
+            using (cache1.QueryContinuous(qry))
+            {
+                // Put from local node.
+                try
+                {
+                    cache1.GetAndPut(PrimaryKey(cache1), Entry(1));
+
+                    Assert.Fail("Should not reach this place.");
+                }
+                catch (IgniteException)
+                {
+                    // No-op.
+                }
+                catch (Exception)
+                {
+                    Assert.Fail("Unexpected error.");
+                }
+
+                // Put from remote node.
+                try
+                {
+                    cache1.GetAndPut(PrimaryKey(cache2), Entry(1));
+
+                    Assert.Fail("Should not reach this place.");
+                }
+                catch (IgniteException)
+                {
+                    // No-op.
+                }
+                catch (Exception)
+                {
+                    Assert.Fail("Unexpected error.");
+                }
+            }
+        }
+
+        /// <summary>
+        /// Test portable filter marshalling error.
+        /// </summary>
+        [Test]
+        public void TestFilterMarshalErrorPortable()
+        {
+            CheckFilterMarshalError(true);
+        }
+
+        /// <summary>
+        /// Test serializable filter marshalling error.
+        /// </summary>
+        [Test]
+        public void TestFilterMarshalErrorSerializable()
+        {
+            CheckFilterMarshalError(false);
+        }
+
+        /// <summary>
+        /// Check filter marshal error handling.
+        /// </summary>
+        /// <param name="portable">Portable flag.</param>
+        private void CheckFilterMarshalError(bool portable)
+        {
+            AbstractFilter<PortableEntry>.marshErr = true;
+
+            ICacheEntryEventListener<int, PortableEntry> lsnr = new Listener<PortableEntry>();
+            ICacheEntryEventFilter<int, PortableEntry> filter =
+                portable ? (AbstractFilter<PortableEntry>)new PortableFilter() : new SerializableFilter();
+
+            ContinuousQuery<int, PortableEntry> qry = new ContinuousQuery<int, PortableEntry>(lsnr, filter);
+
+            Assert.Throws<Exception>(() =>
+            {
+                using (cache1.QueryContinuous(qry))
+                {
+                    // No-op.
+                }
+            });
+        }
+
+        /// <summary>
+        /// Test non-serializable filter error.
+        /// </summary>
+        [Test]
+        public void TestFilterNonSerializable()
+        {
+            CheckFilterNonSerializable(false);
+        }
+
+        /// <summary>
+        /// Test non-serializable filter behavior.
+        /// </summary>
+        /// <param name="loc"></param>
+        protected void CheckFilterNonSerializable(bool loc)
+        {
+            AbstractFilter<PortableEntry>.unmarshErr = true;
+
+            ICacheEntryEventListener<int, PortableEntry> lsnr = new Listener<PortableEntry>();
+            ICacheEntryEventFilter<int, PortableEntry> filter = new LocalFilter();
+
+            ContinuousQuery<int, PortableEntry> qry = loc
+                ? new ContinuousQuery<int, PortableEntry>(lsnr, filter, true)
+                : new ContinuousQuery<int, PortableEntry>(lsnr, filter);
+
+            if (loc)
+            {
+                using (cache1.QueryContinuous(qry))
+                {
+                    // Local put must be fine.
+                    int key1 = PrimaryKey(cache1);
+                    cache1.GetAndPut(key1, Entry(key1));
+                    CheckFilterSingle(key1, null, Entry(key1));
+                }
+            }
+            else
+            {
+                Assert.Throws<SerializationException>(() =>
+                {
+                    using (cache1.QueryContinuous(qry))
+                    {
+                        // No-op.
+                    }
+                });
+            }
+        }
+
+        /// <summary>
+        /// Test portable filter unmarshalling error.
+        /// </summary>
+        [Ignore("IGNITE-521")]
+        [Test]
+        public void TestFilterUnmarshalErrorPortable()
+        {
+            CheckFilterUnmarshalError(true);
+        }
+        
+        /// <summary>
+        /// Test serializable filter unmarshalling error.
+        /// </summary>
+        [Ignore("IGNITE-521")]
+        [Test]
+        public void TestFilterUnmarshalErrorSerializable()
+        {
+            CheckFilterUnmarshalError(false);
+        }
+
+        /// <summary>
+        /// Check filter unmarshal error handling.
+        /// </summary>
+        /// <param name="portable">Portable flag.</param>
+        private void CheckFilterUnmarshalError(bool portable)
+        {
+            AbstractFilter<PortableEntry>.unmarshErr = true;
+
+            ICacheEntryEventListener<int, PortableEntry> lsnr = new Listener<PortableEntry>();
+            ICacheEntryEventFilter<int, PortableEntry> filter =
+                portable ? (AbstractFilter<PortableEntry>)new PortableFilter() : new SerializableFilter();
+
+            ContinuousQuery<int, PortableEntry> qry = new ContinuousQuery<int, PortableEntry>(lsnr, filter);
+
+            using (cache1.QueryContinuous(qry))
+            {
+                // Local put must be fine.
+                int key1 = PrimaryKey(cache1);
+                cache1.GetAndPut(key1, Entry(key1));
+                CheckFilterSingle(key1, null, Entry(key1));
+                
+                // Remote put must fail.
+                try
+                {
+                    cache1.GetAndPut(PrimaryKey(cache2), Entry(1));
+
+                    Assert.Fail("Should not reach this place.");
+                }
+                catch (IgniteException)
+                {
+                    // No-op.
+                }
+                catch (Exception)
+                {
+                    Assert.Fail("Unexpected error.");
+                }
+            }
+        }
+
+        /// <summary>
+        /// Test Ignite injection into filters.
+        /// </summary>
+        [Test]
+        public void TestFilterInjection()
+        {
+            Listener<PortableEntry> cb = new Listener<PortableEntry>();
+            PortableFilter filter = new PortableFilter();
+
+            Assert.IsNull(filter.ignite);
+
+            using (cache1.QueryContinuous(new ContinuousQuery<int, PortableEntry>(cb, filter)))
+            {
+                // Local injection.
+                Assert.IsNotNull(filter.ignite);
+
+                // Remote injection.
+                cache1.GetAndPut(PrimaryKey(cache2), Entry(1));
+
+                FilterEvent evt;
+
+                Assert.IsTrue(FILTER_EVTS.TryTake(out evt, 500));
+
+                Assert.IsNotNull(evt.ignite);
+            }
+        }
+
+
+        /// <summary>
+        /// Test "keep-portable" scenario.
+        /// </summary>
+        [Test]
+        public void TestKeepPortable()
+        {
+            var cache = cache1.WithKeepPortable<int, IPortableObject>();
+
+            ContinuousQuery<int, IPortableObject> qry = new ContinuousQuery<int, IPortableObject>(
+                    new Listener<IPortableObject>(), new KeepPortableFilter());
+
+            using (cache.QueryContinuous(qry))
+            {
+                // 1. Local put.
+                cache1.GetAndPut(PrimaryKey(cache1), Entry(1));
+
+                CallbackEvent cbEvt;
+                FilterEvent filterEvt;
+
+                Assert.IsTrue(FILTER_EVTS.TryTake(out filterEvt, 500));
+                Assert.AreEqual(PrimaryKey(cache1), filterEvt.entry.Key);
+                Assert.AreEqual(null, filterEvt.entry.OldValue);
+                Assert.AreEqual(Entry(1), (filterEvt.entry.Value as IPortableObject)
+                    .Deserialize<PortableEntry>());
+
+                Assert.IsTrue(CB_EVTS.TryTake(out cbEvt, 500));
+                Assert.AreEqual(1, cbEvt.entries.Count);
+                Assert.AreEqual(PrimaryKey(cache1), cbEvt.entries.First().Key);
+                Assert.AreEqual(null, cbEvt.entries.First().OldValue);
+                Assert.AreEqual(Entry(1), (cbEvt.entries.First().Value as IPortableObject)
+                    .Deserialize<PortableEntry>());
+
+                // 2. Remote put.
+                cache1.GetAndPut(PrimaryKey(cache2), Entry(2));
+
+                Assert.IsTrue(FILTER_EVTS.TryTake(out filterEvt, 500));
+                Assert.AreEqual(PrimaryKey(cache2), filterEvt.entry.Key);
+                Assert.AreEqual(null, filterEvt.entry.OldValue);
+                Assert.AreEqual(Entry(2), (filterEvt.entry.Value as IPortableObject)
+                    .Deserialize<PortableEntry>());
+
+                Assert.IsTrue(CB_EVTS.TryTake(out cbEvt, 500));
+                Assert.AreEqual(1, cbEvt.entries.Count);
+                Assert.AreEqual(PrimaryKey(cache2), cbEvt.entries.First().Key);
+                Assert.AreEqual(null, cbEvt.entries.First().OldValue);
+                Assert.AreEqual(Entry(2),
+                    (cbEvt.entries.First().Value as IPortableObject).Deserialize<PortableEntry>());
+            }
+        }
+
+        /// <summary>
+        /// Test whether buffer size works fine.
+        /// </summary>
+        [Test]
+        public void TestBufferSize()
+        {
+            // Put two remote keys in advance.
+            List<int> rmtKeys = PrimaryKeys(cache2, 2);
+
+            ContinuousQuery<int, PortableEntry> qry = new ContinuousQuery<int, PortableEntry>(new Listener<PortableEntry>());
+
+            qry.BufferSize = 2;
+            qry.TimeInterval = TimeSpan.FromMilliseconds(1000000);
+
+            using (cache1.QueryContinuous(qry))
+            {
+                qry.BufferSize = 2;
+
+                cache1.GetAndPut(rmtKeys[0], Entry(rmtKeys[0]));
+
+                CheckNoCallback(100);
+                
+                cache1.GetAndPut(rmtKeys[1], Entry(rmtKeys[1]));
+                
+                CallbackEvent evt;
+
+                Assert.IsTrue(CB_EVTS.TryTake(out evt, 1000));
+
+                Assert.AreEqual(2, evt.entries.Count);
+
+                var entryRmt0 = evt.entries.Single(entry => { return entry.Key.Equals(rmtKeys[0]); });
+                var entryRmt1 = evt.entries.Single(entry => { return entry.Key.Equals(rmtKeys[1]); });
+
+                Assert.AreEqual(rmtKeys[0], entryRmt0.Key);
+                Assert.IsNull(entryRmt0.OldValue);
+                Assert.AreEqual(Entry(rmtKeys[0]), entryRmt0.Value);
+
+                Assert.AreEqual(rmtKeys[1], entryRmt1.Key);
+                Assert.IsNull(entryRmt1.OldValue);
+                Assert.AreEqual(Entry(rmtKeys[1]), entryRmt1.Value);
+            }
+
+            cache1.Remove(rmtKeys[0]);
+            cache1.Remove(rmtKeys[1]);
+        }
+
+        /// <summary>
+        /// Test whether timeout works fine.
+        /// </summary>
+        [Test]
+        public void TestTimeout()
+        {
+            int key1 = PrimaryKey(cache1);
+            int key2 = PrimaryKey(cache2);
+
+            ContinuousQuery<int, PortableEntry> qry =
+                new ContinuousQuery<int, PortableEntry>(new Listener<PortableEntry>());
+
+            qry.BufferSize = 2;
+            qry.TimeInterval = TimeSpan.FromMilliseconds(500);
+
+            using (cache1.QueryContinuous(qry))
+            {
+                // Put from local node.
+                cache1.GetAndPut(key1, Entry(key1));
+                CheckCallbackSingle(key1, null, Entry(key1));
+
+                // Put from remote node.
+                cache1.GetAndPut(key2, Entry(key2));
+                CheckNoCallback(100);
+                CheckCallbackSingle(key2, null, Entry(key2), 1000);
+            }
+        }
+
+        /// <summary>
+        /// Test whether nested Ignite API call from callback works fine.
+        /// </summary>
+        [Test]
+        public void TestNestedCallFromCallback()
+        {
+            var cache = cache1.WithKeepPortable<int, IPortableObject>();
+
+            int key = PrimaryKey(cache1);
+
+            NestedCallListener cb = new NestedCallListener();
+
+            using (cache.QueryContinuous(new ContinuousQuery<int, IPortableObject>(cb)))
+            {
+                cache1.GetAndPut(key, Entry(key));
+
+                cb.countDown.Wait();
+            }
+
+            cache.Remove(key);
+        }
+
+        /// <summary>
+        /// Tests the initial query.
+        /// </summary>
+        [Test]
+        public void TestInitialQuery()
+        {
+            // Scan query, GetAll
+            TestInitialQuery(new ScanQuery<int, PortableEntry>(new InitialQueryScanFilter()), cur => cur.GetAll());
+
+            // Scan query, iterator
+            TestInitialQuery(new ScanQuery<int, PortableEntry>(new InitialQueryScanFilter()), cur => cur.ToList());
+
+            // Sql query, GetAll
+            TestInitialQuery(new SqlQuery(typeof(PortableEntry), "val < 33"), cur => cur.GetAll());
+            
+            // Sql query, iterator
+            TestInitialQuery(new SqlQuery(typeof(PortableEntry), "val < 33"), cur => cur.ToList());
+
+            // Text query, GetAll
+            TestInitialQuery(new TextQuery(typeof(PortableEntry), "1*"), cur => cur.GetAll());
+            
+            // Text query, iterator
+            TestInitialQuery(new TextQuery(typeof(PortableEntry), "1*"), cur => cur.ToList());
+
+            // Test exception: invalid initial query
+            var ex = Assert.Throws<IgniteException>(
+                () => TestInitialQuery(new TextQuery(typeof (PortableEntry), "*"), cur => cur.GetAll()));
+
+            Assert.AreEqual("Cannot parse '*': '*' or '?' not allowed as first character in WildcardQuery", ex.Message);
+        }
+
+        /// <summary>
+        /// Tests the initial query.
+        /// </summary>
+        private void TestInitialQuery(QueryBase initialQry, Func<IQueryCursor<ICacheEntry<int, PortableEntry>>, 
+            IEnumerable<ICacheEntry<int, PortableEntry>>> getAllFunc)
+        {
+            var qry = new ContinuousQuery<int, PortableEntry>(new Listener<PortableEntry>());
+
+            cache1.Put(11, Entry(11));
+            cache1.Put(12, Entry(12));
+            cache1.Put(33, Entry(33));
+
+            try
+            {
+                IContinuousQueryHandle<ICacheEntry<int, PortableEntry>> contQry;
+                
+                using (contQry = cache1.QueryContinuous(qry, initialQry))
+                {
+                    // Check initial query
+                    var initialEntries =
+                        getAllFunc(contQry.GetInitialQueryCursor()).Distinct().OrderBy(x => x.Key).ToList();
+
+                    Assert.Throws<InvalidOperationException>(() => contQry.GetInitialQueryCursor());
+
+                    Assert.AreEqual(2, initialEntries.Count);
+
+                    for (int i = 0; i < initialEntries.Count; i++)
+                    {
+                        Assert.AreEqual(i + 11, initialEntries[i].Key);
+                        Assert.AreEqual(i + 11, initialEntries[i].Value.val);
+                    }
+
+                    // Check continuous query
+                    cache1.Put(44, Entry(44));
+                    CheckCallbackSingle(44, null, Entry(44));
+                }
+
+                Assert.Throws<ObjectDisposedException>(() => contQry.GetInitialQueryCursor());
+
+                contQry.Dispose();  // multiple dispose calls are ok
+            }
+            finally
+            {
+                cache1.Clear();
+            }
+        }
+
+        /// <summary>
+        /// Check single filter event.
+        /// </summary>
+        /// <param name="expKey">Expected key.</param>
+        /// <param name="expOldVal">Expected old value.</param>
+        /// <param name="expVal">Expected value.</param>
+        private void CheckFilterSingle(int expKey, PortableEntry expOldVal, PortableEntry expVal)
+        {
+            CheckFilterSingle(expKey, expOldVal, expVal, 1000);
+        }
+
+        /// <summary>
+        /// Check single filter event.
+        /// </summary>
+        /// <param name="expKey">Expected key.</param>
+        /// <param name="expOldVal">Expected old value.</param>
+        /// <param name="expVal">Expected value.</param>
+        /// <param name="timeout">Timeout.</param>
+        private void CheckFilterSingle(int expKey, PortableEntry expOldVal, PortableEntry expVal, int timeout)
+        {
+            FilterEvent evt;
+
+            Assert.IsTrue(FILTER_EVTS.TryTake(out evt, timeout));
+
+            Assert.AreEqual(expKey, evt.entry.Key);
+            Assert.AreEqual(expOldVal, evt.entry.OldValue);
+            Assert.AreEqual(expVal, evt.entry.Value);
+        }
+
+        /// <summary>
+        /// Ensure that no filter events are logged.
+        /// </summary>
+        /// <param name="timeout">Timeout.</param>
+        private void CheckNoFilter(int timeout)
+        {
+            FilterEvent evt;
+
+            Assert.IsFalse(FILTER_EVTS.TryTake(out evt, timeout));
+        }
+
+        /// <summary>
+        /// Check single callback event.
+        /// </summary>
+        /// <param name="expKey">Expected key.</param>
+        /// <param name="expOldVal">Expected old value.</param>
+        /// <param name="expVal">Expected new value.</param>
+        private void CheckCallbackSingle(int expKey, PortableEntry expOldVal, PortableEntry expVal)
+        {
+            CheckCallbackSingle(expKey, expOldVal, expVal, 1000);
+        }
+
+        /// <summary>
+        /// Check single callback event.
+        /// </summary>
+        /// <param name="expKey">Expected key.</param>
+        /// <param name="expOldVal">Expected old value.</param>
+        /// <param name="expVal">Expected new value.</param>
+        /// <param name="timeout">Timeout.</param>
+        private void CheckCallbackSingle(int expKey, PortableEntry expOldVal, PortableEntry expVal, int timeout)
+        {
+            CallbackEvent evt;
+
+            Assert.IsTrue(CB_EVTS.TryTake(out evt, timeout));
+
+            Assert.AreEqual(1, evt.entries.Count);
+
+            Assert.AreEqual(expKey, evt.entries.First().Key);
+            Assert.AreEqual(expOldVal, evt.entries.First().OldValue);
+            Assert.AreEqual(expVal, evt.entries.First().Value);
+        }
+
+        /// <summary>
+        /// Ensure that no callback events are logged.
+        /// </summary>
+        /// <param name="timeout">Timeout.</param>
+        private void CheckNoCallback(int timeout)
+        {
+            CallbackEvent evt;
+
+            Assert.IsFalse(CB_EVTS.TryTake(out evt, timeout));
+        }
+
+        /// <summary>
+        /// Craate entry.
+        /// </summary>
+        /// <param name="val">Value.</param>
+        /// <returns>Entry.</returns>
+        private static PortableEntry Entry(int val)
+        {
+            return new PortableEntry(val);
+        }
+
+        /// <summary>
+        /// Get primary key for cache.
+        /// </summary>
+        /// <param name="cache">Cache.</param>
+        /// <returns>Primary key.</returns>
+        private static int PrimaryKey<T>(ICache<int, T> cache)
+        {
+            return PrimaryKeys(cache, 1)[0];
+        }
+
+        /// <summary>
+        /// Get primary keys for cache.
+        /// </summary>
+        /// <param name="cache">Cache.</param>
+        /// <param name="cnt">Amount of keys.</param>
+        /// <param name="startFrom">Value to start from.</param>
+        /// <returns></returns>
+        private static List<int> PrimaryKeys<T>(ICache<int, T> cache, int cnt, int startFrom = 0)
+        {
+            IClusterNode node = cache.Ignite.Cluster.LocalNode;
+
+            ICacheAffinity aff = cache.Ignite.Affinity(cache.Name);
+
+            List<int> keys = new List<int>(cnt);
+
+            for (int i = startFrom; i < startFrom + 100000; i++)
+            {
+                if (aff.IsPrimary(node, i))
+                {
+                    keys.Add(i);
+
+                    if (keys.Count == cnt)
+                        return keys;
+                }
+            }
+
+            Assert.Fail("Failed to find " + cnt + " primary keys.");
+
+            return null;
+        }
+
+        /// <summary>
+        /// Portable entry.
+        /// </summary>
+        public class PortableEntry
+        {
+            /** Value. */
+            public readonly int val;
+
+            /** <inheritDot /> */
+            public override int GetHashCode()
+            {
+                return val;
+            }
+
+            /// <summary>
+            /// Constructor.
+            /// </summary>
+            /// <param name="val">Value.</param>
+            public PortableEntry(int val)
+            {
+                this.val = val;
+            }
+
+            /** <inheritDoc /> */
+            public override bool Equals(object obj)
+            {
+                return obj != null && obj is PortableEntry && ((PortableEntry)obj).val == val;
+            }
+        }
+
+        /// <summary>
+        /// Abstract filter.
+        /// </summary>
+        [Serializable]
+        public abstract class AbstractFilter<V> : ICacheEntryEventFilter<int, V>
+        {
+            /** Result. */
+            public static volatile bool res = true;
+
+            /** Throw error on invocation. */
+            public static volatile bool err;
+
+            /** Throw error during marshalling. */
+            public static volatile bool marshErr;
+
+            /** Throw error during unmarshalling. */
+            public static volatile bool unmarshErr;
+
+            /** Grid. */
+            [InstanceResource]
+            public IIgnite ignite;
+
+            /** <inheritDoc /> */
+            public bool Evaluate(ICacheEntryEvent<int, V> evt)
+            {
+                if (err)
+                    throw new Exception("Filter error.");
+
+                FILTER_EVTS.Add(new FilterEvent(ignite,
+                    CQU.CreateEvent<object, object>(evt.Key, evt.OldValue, evt.Value)));
+
+                return res;
+            }
+        }
+
+        /// <summary>
+        /// Filter which cannot be serialized.
+        /// </summary>
+        public class LocalFilter : AbstractFilter<PortableEntry>
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Portable filter.
+        /// </summary>
+        public class PortableFilter : AbstractFilter<PortableEntry>, IPortableMarshalAware
+        {
+            /** <inheritDoc /> */
+            public void WritePortable(IPortableWriter writer)
+            {
+                if (marshErr)
+                    throw new Exception("Filter marshalling error.");
+            }
+
+            /** <inheritDoc /> */
+            public void ReadPortable(IPortableReader reader)
+            {
+                if (unmarshErr)
+                    throw new Exception("Filter unmarshalling error.");
+            }
+        }
+
+        /// <summary>
+        /// Serializable filter.
+        /// </summary>
+        [Serializable]
+        public class SerializableFilter : AbstractFilter<PortableEntry>, ISerializable
+        {
+            /// <summary>
+            /// Constructor.
+            /// </summary>
+            public SerializableFilter()
+            {
+                // No-op.
+            }
+
+            /// <summary>
+            /// Serialization constructor.
+            /// </summary>
+            /// <param name="info">Info.</param>
+            /// <param name="context">Context.</param>
+            protected SerializableFilter(SerializationInfo info, StreamingContext context)
+            {
+                if (unmarshErr)
+                    throw new Exception("Filter unmarshalling error.");
+            }
+
+            /** <inheritDoc /> */
+            public void GetObjectData(SerializationInfo info, StreamingContext context)
+            {
+                if (marshErr)
+                    throw new Exception("Filter marshalling error.");
+            }
+        }
+
+        /// <summary>
+        /// Filter for "keep-portable" scenario.
+        /// </summary>
+        public class KeepPortableFilter : AbstractFilter<IPortableObject>
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Listener.
+        /// </summary>
+        public class Listener<V> : ICacheEntryEventListener<int, V>
+        {
+            [InstanceResource]
+            public IIgnite ignite;
+            
+            /** <inheritDoc /> */
+            public void OnEvent(IEnumerable<ICacheEntryEvent<int, V>> evts)
+            {
+                ICollection<ICacheEntryEvent<object, object>> entries0 =
+                    new List<ICacheEntryEvent<object, object>>();
+
+                foreach (ICacheEntryEvent<int, V> evt in evts)
+                    entries0.Add(CQU.CreateEvent<object, object>(evt.Key, evt.OldValue, evt.Value));
+
+                CB_EVTS.Add(new CallbackEvent(entries0));
+            }
+        }
+
+        /// <summary>
+        /// Listener with nested Ignite API call.
+        /// </summary>
+        public class NestedCallListener : ICacheEntryEventListener<int, IPortableObject>
+        {
+            /** Event. */
+            public readonly CountdownEvent countDown = new CountdownEvent(1);
+
+            public void OnEvent(IEnumerable<ICacheEntryEvent<int, IPortableObject>> evts)
+            {
+                foreach (ICacheEntryEvent<int, IPortableObject> evt in evts)
+                {
+                    IPortableObject val = evt.Value;
+
+                    IPortableMetadata meta = val.Metadata();
+
+                    Assert.AreEqual(typeof(PortableEntry).Name, meta.TypeName);
+                }
+
+                countDown.Signal();
+            }
+        }
+
+        /// <summary>
+        /// Filter event.
+        /// </summary>
+        public class FilterEvent
+        {
+            /** Grid. */
+            public IIgnite ignite;
+
+            /** Entry. */
+            public ICacheEntryEvent<object, object> entry;
+
+            /// <summary>
+            /// Constructor.
+            /// </summary>
+            /// <param name="ignite">Grid.</param>
+            /// <param name="entry">Entry.</param>
+            public FilterEvent(IIgnite ignite, ICacheEntryEvent<object, object> entry)
+            {
+                this.ignite = ignite;
+                this.entry = entry;
+            }
+        }
+
+        /// <summary>
+        /// Callbakc event.
+        /// </summary>
+        public class CallbackEvent
+        {
+            /** Entries. */
+            public ICollection<ICacheEntryEvent<object, object>> entries;
+
+            /// <summary>
+            /// Constructor.
+            /// </summary>
+            /// <param name="entries">Entries.</param>
+            public CallbackEvent(ICollection<ICacheEntryEvent<object, object>> entries)
+            {
+                this.entries = entries;
+            }
+        }
+
+        /// <summary>
+        /// ScanQuery filter for InitialQuery test.
+        /// </summary>
+        [Serializable]
+        private class InitialQueryScanFilter : ICacheEntryFilter<int, PortableEntry>
+        {
+            /** <inheritdoc /> */
+            public bool Invoke(ICacheEntry<int, PortableEntry> entry)
+            {
+                return entry.Key < 33;
+            }
+        }
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryAtomicBackupTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryAtomicBackupTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryAtomicBackupTest.cs
new file mode 100644
index 0000000..ac44f10
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryAtomicBackupTest.cs
@@ -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.
+ */
+
+namespace Apache.Ignite.Core.Tests.Cache.Query.Continuous
+{
+    /// <summary>
+    /// Continuous query tests for ATOMIC cache with backups.
+    /// </summary>
+    public class ContinuousQueryAtomiclBackupTest : ContinuousQueryAbstractTest
+    {
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        public ContinuousQueryAtomiclBackupTest() : base(CACHE_ATOMIC_BACKUP)
+        {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryAtomicNoBackupTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryAtomicNoBackupTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryAtomicNoBackupTest.cs
new file mode 100644
index 0000000..8e1a18f
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryAtomicNoBackupTest.cs
@@ -0,0 +1,34 @@
+/*
+ * 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.Cache.Query.Continuous
+{
+    /// <summary>
+    /// Continuous query tests for ATOMIC cache with no backups.
+    /// </summary>
+    public class ContinuousQueryAtomiclNoBackupTest : ContinuousQueryNoBackupAbstractTest
+    {
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        public ContinuousQueryAtomiclNoBackupTest()
+            : base(CACHE_ATOMIC_NO_BACKUP)
+        {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryNoBackupAbstractTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryNoBackupAbstractTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryNoBackupAbstractTest.cs
new file mode 100644
index 0000000..aa7d627
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryNoBackupAbstractTest.cs
@@ -0,0 +1,72 @@
+/*
+ * 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.Cache.Query.Continuous
+{
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests for ocntinuous query when there are no backups.
+    /// </summary>
+    public abstract class ContinuousQueryNoBackupAbstractTest : ContinuousQueryAbstractTest
+    {
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="cacheName">Cache name.</param>
+        protected ContinuousQueryNoBackupAbstractTest(string cacheName) : base(cacheName)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Test regular callback operations for local query.
+        /// </summary>
+        [Test]
+        public void TestCallbackLocal()
+        {
+            CheckCallback(true);
+        }
+
+        /// <summary>
+        /// Test portable filter logic.
+        /// </summary>
+        [Test]
+        public void TestFilterPortableLocal()
+        {
+            CheckFilter(true, true);
+        }
+
+        /// <summary>
+        /// Test serializable filter logic.
+        /// </summary>
+        [Test]
+        public void TestFilterSerializableLocal()
+        {
+            CheckFilter(false, true);
+        }
+
+        /// <summary>
+        /// Test non-serializable filter for local query.
+        /// </summary>
+        [Test]
+        public void TestFilterNonSerializableLocal()
+        {
+            CheckFilterNonSerializable(true);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryTransactionalBackupTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryTransactionalBackupTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryTransactionalBackupTest.cs
new file mode 100644
index 0000000..08ae88c
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryTransactionalBackupTest.cs
@@ -0,0 +1,34 @@
+/*
+ * 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.Cache.Query.Continuous
+{
+    /// <summary>
+    /// Continuous query tests for TRANSACTIONAL cache with backups.
+    /// </summary>
+    public class ContinuousQueryTransactionalBackupTest : ContinuousQueryAbstractTest
+    {
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        public ContinuousQueryTransactionalBackupTest()
+            : base(CACHE_TX_BACKUP)
+        {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryTransactionalNoBackupTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryTransactionalNoBackupTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryTransactionalNoBackupTest.cs
new file mode 100644
index 0000000..685f7b4
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryTransactionalNoBackupTest.cs
@@ -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.
+ */
+
+namespace Apache.Ignite.Core.Tests.Cache.Query.Continuous
+{
+    /// <summary>
+    /// Continuous query tests for TRANSACTIONAL cache with no backups.
+    /// </summary>
+    public class ContinuousQueryTransactionalNoBackupTest : ContinuousQueryNoBackupAbstractTest
+    {
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        public ContinuousQueryTransactionalNoBackupTest() : base(CACHE_TX_NO_BACKUP)
+        {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheParallelLoadStoreTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheParallelLoadStoreTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheParallelLoadStoreTest.cs
new file mode 100644
index 0000000..33eec7b
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheParallelLoadStoreTest.cs
@@ -0,0 +1,110 @@
+/*
+ * 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.Cache.Store
+{
+    using System;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Portable;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests for GridCacheParallelLoadStoreAdapter.
+    /// </summary>
+    public class CacheParallelLoadStoreTest
+    {
+        // object store name
+        private const string ObjectStoreCacheName = "object_store_parallel";
+
+        /// <summary>
+        /// Set up test class.
+        /// </summary>
+        [TestFixtureSetUp]
+        public virtual void BeforeTests()
+        {
+            TestUtils.KillProcesses();
+            TestUtils.JvmDebug = true;
+
+            Ignition.Start(new IgniteConfiguration
+            {
+                JvmClasspath = TestUtils.CreateTestClasspath(),
+                JvmOptions = TestUtils.TestJavaOptions(),
+                SpringConfigUrl = "config\\native-client-test-cache-parallel-store.xml",
+                PortableConfiguration = new PortableConfiguration
+                {
+                    Types = new[] {typeof (CacheTestParallelLoadStore.Record).FullName}
+                }
+            });
+        }
+
+        /// <summary>
+        /// Tear down test class.
+        /// </summary>
+        [TestFixtureTearDown]
+        public virtual void AfterTests()
+        {
+            Ignition.StopAll(true);
+        }
+
+        /// <summary>
+        /// Test setup.
+        /// </summary>
+        [SetUp]
+        public void BeforeTest()
+        {
+            Console.WriteLine("Test started: " + TestContext.CurrentContext.Test.Name);
+        }
+
+        /// <summary>
+        /// Tests the LoadCache.
+        /// </summary>
+        [Test]
+        public void TestLoadCache()
+        {
+            var cache = GetCache();
+
+            Assert.AreEqual(0, cache.Size());
+
+            const int minId = 113;
+            const int expectedItemCount = CacheTestParallelLoadStore.InputDataLength - minId;
+
+            CacheTestParallelLoadStore.ResetCounters();
+
+            cache.LocalLoadCache(null, minId);
+
+            Assert.AreEqual(expectedItemCount, cache.Size());
+
+            // check items presence; increment by 100 to speed up the test
+            for (var i = minId; i < expectedItemCount; i += 100)
+            {
+                var rec = cache.Get(i);
+                Assert.AreEqual(i, rec.Id);
+            }
+
+            // check that items were processed in parallel
+            Assert.GreaterOrEqual(CacheTestParallelLoadStore.UniqueThreadCount, Environment.ProcessorCount);
+        }
+
+        /// <summary>
+        /// Gets the cache.
+        /// </summary>
+        private static ICache<int, CacheTestParallelLoadStore.Record> GetCache()
+        {
+            return Ignition.GetIgnite().Cache<int, CacheTestParallelLoadStore.Record>(ObjectStoreCacheName);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreSessionTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreSessionTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreSessionTest.cs
new file mode 100644
index 0000000..bc55901
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreSessionTest.cs
@@ -0,0 +1,285 @@
+/*
+ * 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.Cache.Store
+{
+    using System;
+    using System.Collections.Concurrent;
+    using System.Collections.Generic;
+    using System.Linq;
+    using Apache.Ignite.Core.Cache.Store;
+    using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.Resource;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests for store session.
+    /// </summary>
+    public class CacheStoreSessionTest
+    {
+        /** Grid name. */
+        private const string IgniteName = "grid";
+
+        /** Cache 1 name. */
+        private const string Cache1 = "cache1";
+
+        /** Cache 2 name. */
+        private const string Cache2 = "cache2";
+
+        /** Operations. */
+        private static ConcurrentBag<ICollection<Operation>> _dumps;
+
+        /// <summary>
+        /// Set up routine.
+        /// </summary>
+        [TestFixtureSetUp]
+        public virtual void BeforeTests()
+        {
+            //TestUtils.JVM_DEBUG = true;
+
+            TestUtils.KillProcesses();
+
+            TestUtils.JvmDebug = true;
+
+            IgniteConfigurationEx cfg = new IgniteConfigurationEx
+            {
+                GridName = IgniteName,
+                JvmClasspath = TestUtils.CreateTestClasspath(),
+                JvmOptions = TestUtils.TestJavaOptions(),
+                SpringConfigUrl = @"config\cache\store\cache-store-session.xml"
+            };
+
+
+            Ignition.Start(cfg);
+        }
+
+        /// <summary>
+        /// Tear down routine.
+        /// </summary>
+        [TestFixtureTearDown]
+        public virtual void AfterTests()
+        {
+            Ignition.StopAll(true);
+        }
+        
+        /// <summary>
+        /// Test basic session API.
+        /// </summary>
+        [Test]
+        public void TestSession()
+        {
+            _dumps = new ConcurrentBag<ICollection<Operation>>();
+
+            var ignite = Ignition.GetIgnite(IgniteName);
+
+            var cache1 = Ignition.GetIgnite(IgniteName).Cache<int, int>(Cache1);
+            var cache2 = Ignition.GetIgnite(IgniteName).Cache<int, int>(Cache2);
+
+            // 1. Test rollback.
+            using (var tx = ignite.Transactions.TxStart())
+            {
+                cache1.Put(1, 1);
+                cache2.Put(2, 2);
+
+                tx.Rollback();
+            }
+
+            Assert.AreEqual(1, _dumps.Count);
+            var ops = _dumps.First();
+            Assert.AreEqual(1, ops.Count);
+
+            Assert.AreEqual(1, ops.Count(op => op.Type == OperationType.SesEnd && !op.Commit));
+
+            _dumps = new ConcurrentBag<ICollection<Operation>>();
+
+            // 2. Test puts.
+            using (var tx = ignite.Transactions.TxStart())
+            {
+                cache1.Put(1, 1);
+                cache2.Put(2, 2);
+
+                tx.Commit();
+            }
+
+            Assert.AreEqual(1, _dumps.Count);
+            ops = _dumps.First();
+            Assert.AreEqual(3, ops.Count);
+
+            Assert.AreEqual(1, ops.Count(op => op.Type == OperationType.Write && Cache1.Equals(op.CacheName) && 1.Equals(op.Key) && 1.Equals(op.Value)));
+            Assert.AreEqual(1, ops.Count(op => op.Type == OperationType.Write && Cache2.Equals(op.CacheName) && 2.Equals(op.Key) && 2.Equals(op.Value)));
+            Assert.AreEqual(1, ops.Count(op => op.Type == OperationType.SesEnd && op.Commit));
+
+            _dumps = new ConcurrentBag<ICollection<Operation>>();
+
+            // 3. Test removes.
+            using (var tx = ignite.Transactions.TxStart())
+            {
+                cache1.Remove(1);
+                cache2.Remove(2);
+
+                tx.Commit();
+            }
+
+            Assert.AreEqual(1, _dumps.Count);
+            ops = _dumps.First();
+            Assert.AreEqual(3, ops.Count);
+
+            Assert.AreEqual(1, ops.Count(op => op.Type == OperationType.Delete && Cache1.Equals(op.CacheName) && 1.Equals(op.Key)));
+            Assert.AreEqual(1, ops.Count(op => op.Type == OperationType.Delete && Cache2.Equals(op.CacheName) && 2.Equals(op.Key)));
+            Assert.AreEqual(1, ops.Count(op => op.Type == OperationType.SesEnd && op.Commit));
+        }
+
+        /// <summary>
+        /// Dump operations.
+        /// </summary>
+        /// <param name="dump">Dump.</param>
+        internal static void DumpOperations(ICollection<Operation> dump)
+        {
+            _dumps.Add(dump);
+        }
+
+        /// <summary>
+        /// Test store implementation.
+        /// </summary>
+        public class Store : CacheStoreAdapter
+        {
+            /** Store session. */
+            [StoreSessionResource]
+#pragma warning disable 649
+            private ICacheStoreSession _ses;
+#pragma warning restore 649
+
+            /** <inheritdoc /> */
+            public override object Load(object key)
+            {
+                throw new NotImplementedException();
+            }
+
+            /** <inheritdoc /> */
+            public override void Write(object key, object val)
+            {
+                GetOperations().Add(new Operation(_ses.CacheName, OperationType.Write, (int)key, (int)val));
+            }
+
+            /** <inheritdoc /> */
+            public override void Delete(object key)
+            {
+                GetOperations().Add(new Operation(_ses.CacheName, OperationType.Delete, (int)key, 0));
+            }
+
+            /** <inheritdoc /> */
+            public override void SessionEnd(bool commit)
+            {
+                Operation op = new Operation(_ses.CacheName, OperationType.SesEnd) { Commit = commit };
+
+                ICollection<Operation> ops = GetOperations();
+
+                ops.Add(op);
+
+                DumpOperations(ops);
+            }
+
+            /// <summary>
+            /// Get collection with operations.
+            /// </summary>
+            /// <returns>Operations.</returns>
+            private ICollection<Operation> GetOperations()
+            {
+                object ops;
+
+                if (!_ses.Properties.TryGetValue("ops", out ops))
+                {
+                    ops = new List<Operation>();
+
+                    _ses.Properties["ops"] = ops;
+                }
+
+                return (ICollection<Operation>) ops;
+            } 
+        }
+
+        /// <summary>
+        /// Logged operation.
+        /// </summary>
+        internal class Operation
+        {
+            /// <summary>
+            /// Constructor.
+            /// </summary>
+            /// <param name="cacheName">Cache name.</param>
+            /// <param name="type">Operation type.</param>
+            public Operation(string cacheName, OperationType type)
+            {
+                CacheName = cacheName;
+                Type = type;
+            }
+
+            /// <summary>
+            /// Constructor.
+            /// </summary>
+            /// <param name="cacheName">Cache name.</param>
+            /// <param name="type">Operation type.</param>
+            /// <param name="key">Key.</param>
+            /// <param name="val">Value.</param>
+            public Operation(string cacheName, OperationType type, int key, int val) : this(cacheName, type)
+            {
+                Key = key;
+                Value = val;
+            }
+
+            /// <summary>
+            /// Cache name.
+            /// </summary>
+            public string CacheName { get; set; }
+            
+            /// <summary>
+            /// Operation type.
+            /// </summary>
+            public OperationType Type { get; set; }
+
+            /// <summary>
+            /// Key.
+            /// </summary>
+            public int Key { get; set; }
+
+            /// <summary>
+            /// Value.
+            /// </summary>
+            public int Value { get; set; }
+
+            /// <summary>
+            /// Commit flag.
+            /// </summary>
+            public bool Commit { get; set; }
+        }
+
+        /// <summary>
+        /// Operation types.
+        /// </summary>
+        internal enum OperationType
+        {
+            /** Write. */
+            Write,
+
+            /** Delete. */
+            Delete,
+
+            /** Session end. */
+            SesEnd
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreTest.cs
new file mode 100644
index 0000000..4e5e050
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreTest.cs
@@ -0,0 +1,510 @@
+/*
+ * 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.Cache.Store
+{
+    using System;
+    using System.Collections;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.Portable;
+    using NUnit.Framework;
+
+    /// <summary>
+    ///
+    /// </summary>
+    class Key
+    {
+        private readonly int _idx;
+
+        public Key(int idx)
+        {
+            _idx = idx;
+        }
+
+        public int Index()
+        {
+            return _idx;
+        }
+
+        public override bool Equals(object obj)
+        {
+            if (obj == null || obj.GetType() != GetType())
+                return false;
+
+            Key key = (Key)obj;
+
+            return key._idx == _idx;
+        }
+
+        public override int GetHashCode()
+        {
+            return _idx;
+        }
+    }
+
+    /// <summary>
+    ///
+    /// </summary>
+    class Value
+    {
+        private int _idx;
+
+        public Value(int idx)
+        {
+            _idx = idx;
+        }
+
+        public int Index()
+        {
+            return _idx;
+        }
+    }
+
+    /// <summary>
+    /// Cache entry predicate.
+    /// </summary>
+    [Serializable]
+    public class CacheEntryFilter : ICacheEntryFilter<int, string>
+    {
+        /** <inheritdoc /> */
+        public bool Invoke(ICacheEntry<int, string> entry)
+        {
+            return entry.Key >= 105;
+        }
+    }
+
+    /// <summary>
+    ///
+    /// </summary>
+    public class CacheStoreTest
+    {
+        /** */
+        private const string PortableStoreCacheName = "portable_store";
+
+        /** */
+        private const string ObjectStoreCacheName = "object_store";
+
+        /** */
+        private const string CustomStoreCacheName = "custom_store";
+
+        /** */
+        private const string TemplateStoreCacheName = "template_store*";
+
+        /// <summary>
+        ///
+        /// </summary>
+        [TestFixtureSetUp]
+        public void BeforeTests()
+        {
+            //TestUtils.JVM_DEBUG = true;
+
+            TestUtils.KillProcesses();
+
+            TestUtils.JvmDebug = true;
+
+            IgniteConfigurationEx cfg = new IgniteConfigurationEx();
+
+            cfg.GridName = GridName();
+            cfg.JvmClasspath = TestUtils.CreateTestClasspath();
+            cfg.JvmOptions = TestUtils.TestJavaOptions();
+            cfg.SpringConfigUrl = "config\\native-client-test-cache-store.xml";
+
+            PortableConfiguration portCfg = new PortableConfiguration();
+
+            portCfg.Types = new List<string> { typeof(Key).FullName, typeof(Value).FullName };
+
+            cfg.PortableConfiguration = portCfg;
+
+            Ignition.Start(cfg);
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        [TestFixtureTearDown]
+        public virtual void AfterTests()
+        {
+            Ignition.StopAll(true);
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        [SetUp]
+        public void BeforeTest()
+        {
+            Console.WriteLine("Test started: " + TestContext.CurrentContext.Test.Name);
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        [TearDown]
+        public void AfterTest()
+        {
+            var cache = Cache();
+
+            cache.Clear();
+
+            Assert.IsTrue(cache.IsEmpty, "Cache is not empty: " + cache.Size());
+
+            CacheTestStore.Reset();
+
+            Console.WriteLine("Test finished: " + TestContext.CurrentContext.Test.Name);
+        }
+
+        [Test]
+        public void TestLoadCache()
+        {
+            var cache = Cache();
+
+            Assert.AreEqual(0, cache.Size());
+
+            cache.LoadCache(new CacheEntryFilter(), 100, 10);
+
+            Assert.AreEqual(5, cache.Size());
+
+            for (int i = 105; i < 110; i++)
+                Assert.AreEqual("val_" + i, cache.Get(i));
+        }
+
+        [Test]
+        public void TestLocalLoadCache()
+        {
+            var cache = Cache();
+
+            Assert.AreEqual(0, cache.Size());
+
+            cache.LocalLoadCache(new CacheEntryFilter(), 100, 10);
+
+            Assert.AreEqual(5, cache.Size());
+
+            for (int i = 105; i < 110; i++)
+                Assert.AreEqual("val_" + i, cache.Get(i));
+        }
+
+        [Test]
+        public void TestLoadCacheMetadata()
+        {
+            CacheTestStore.LoadObjects = true;
+
+            var cache = Cache();
+
+            Assert.AreEqual(0, cache.Size());
+
+            cache.LocalLoadCache(null, 0, 3);
+
+            Assert.AreEqual(3, cache.Size());
+
+            var meta = cache.WithKeepPortable<Key, IPortableObject>().Get(new Key(0)).Metadata();
+
+            Assert.NotNull(meta);
+
+            Assert.AreEqual("Value", meta.TypeName);
+        }
+
+        [Test]
+        public void TestLoadCacheAsync()
+        {
+            var cache = Cache().WithAsync();
+
+            Assert.AreEqual(0, cache.Size());
+
+            cache.LocalLoadCache(new CacheEntryFilter(), 100, 10);
+
+            var fut = cache.GetFuture<object>();
+
+            fut.Get();
+
+            Assert.IsTrue(fut.IsDone);
+
+            cache.Size();
+            Assert.AreEqual(5, cache.GetFuture<int>().ToTask().Result);
+
+            for (int i = 105; i < 110; i++)
+            {
+                cache.Get(i);
+
+                Assert.AreEqual("val_" + i, cache.GetFuture<string>().ToTask().Result);
+            }
+        }
+
+        [Test]
+        public void TestPutLoad()
+        {
+            var cache = Cache();
+
+            cache.Put(1, "val");
+
+            IDictionary map = StoreMap();
+
+            Assert.AreEqual(1, map.Count);
+
+            cache.LocalEvict(new[] { 1 });
+
+            Assert.AreEqual(0, cache.Size());
+
+            Assert.AreEqual("val", cache.Get(1));
+
+            Assert.AreEqual(1, cache.Size());
+        }
+
+        [Test]
+        public void TestPutLoadPortables()
+        {
+            var cache = PortableStoreCache<int, Value>();
+
+            cache.Put(1, new Value(1));
+
+            IDictionary map = StoreMap();
+
+            Assert.AreEqual(1, map.Count);
+
+            IPortableObject v = (IPortableObject)map[1];
+
+            Assert.AreEqual(1, v.Field<int>("_idx"));
+
+            cache.LocalEvict(new[] { 1 });
+
+            Assert.AreEqual(0, cache.Size());
+
+            Assert.AreEqual(1, cache.Get(1).Index());
+
+            Assert.AreEqual(1, cache.Size());
+        }
+
+        [Test]
+        public void TestPutLoadObjects()
+        {
+            var cache = ObjectStoreCache<int, Value>();
+
+            cache.Put(1, new Value(1));
+
+            IDictionary map = StoreMap();
+
+            Assert.AreEqual(1, map.Count);
+
+            Value v = (Value)map[1];
+
+            Assert.AreEqual(1, v.Index());
+
+            cache.LocalEvict(new[] { 1 });
+
+            Assert.AreEqual(0, cache.Size());
+
+            Assert.AreEqual(1, cache.Get(1).Index());
+
+            Assert.AreEqual(1, cache.Size());
+        }
+
+        [Test]
+        public void TestPutLoadAll()
+        {
+            var putMap = new Dictionary<int, string>();
+
+            for (int i = 0; i < 10; i++)
+                putMap.Add(i, "val_" + i);
+
+            var cache = Cache();
+
+            cache.PutAll(putMap);
+
+            IDictionary map = StoreMap();
+
+            Assert.AreEqual(10, map.Count);
+
+            for (int i = 0; i < 10; i++)
+                Assert.AreEqual("val_" + i, map[i]);
+
+            cache.Clear();
+
+            Assert.AreEqual(0, cache.Size());
+
+            ICollection<int> keys = new List<int>();
+
+            for (int i = 0; i < 10; i++)
+                keys.Add(i);
+
+            IDictionary<int, string> loaded = cache.GetAll(keys);
+
+            Assert.AreEqual(10, loaded.Count);
+
+            for (int i = 0; i < 10; i++)
+                Assert.AreEqual("val_" + i, loaded[i]);
+
+            Assert.AreEqual(10, cache.Size());
+        }
+
+        [Test]
+        public void TestRemove()
+        {
+            var cache = Cache();
+
+            for (int i = 0; i < 10; i++)
+                cache.Put(i, "val_" + i);
+
+            IDictionary map = StoreMap();
+
+            Assert.AreEqual(10, map.Count);
+
+            for (int i = 0; i < 5; i++)
+                cache.Remove(i);
+
+            Assert.AreEqual(5, map.Count);
+
+            for (int i = 5; i < 10; i++)
+                Assert.AreEqual("val_" + i, map[i]);
+        }
+
+        [Test]
+        public void TestRemoveAll()
+        {
+            var cache = Cache();
+
+            for (int i = 0; i < 10; i++)
+                cache.Put(i, "val_" + i);
+
+            IDictionary map = StoreMap();
+
+            Assert.AreEqual(10, map.Count);
+
+            cache.RemoveAll(new List<int> { 0, 1, 2, 3, 4 });
+
+            Assert.AreEqual(5, map.Count);
+
+            for (int i = 5; i < 10; i++)
+                Assert.AreEqual("val_" + i, map[i]);
+        }
+
+        [Test]
+        public void TestTx()
+        {
+            var cache = Cache();
+
+            using (var tx = cache.Ignite.Transactions.TxStart())
+            {
+                CacheTestStore.ExpCommit = true;
+
+                tx.AddMeta("meta", 100);
+
+                cache.Put(1, "val");
+
+                tx.Commit();
+            }
+
+            IDictionary map = StoreMap();
+
+            Assert.AreEqual(1, map.Count);
+
+            Assert.AreEqual("val", map[1]);
+        }
+
+        [Test]
+        public void TestLoadCacheMultithreaded()
+        {
+            CacheTestStore.LoadMultithreaded = true;
+
+            var cache = Cache();
+
+            Assert.AreEqual(0, cache.Size());
+
+            cache.LocalLoadCache(null, 0, null);
+
+            Assert.AreEqual(1000, cache.Size());
+
+            for (int i = 0; i < 1000; i++)
+                Assert.AreEqual("val_" + i, cache.Get(i));
+        }
+
+        [Test]
+        public void TestCustomStoreProperties()
+        {
+            var cache = CustomStoreCache();
+            Assert.IsNotNull(cache);
+
+            Assert.AreEqual(42, CacheTestStore.intProperty);
+            Assert.AreEqual("String value", CacheTestStore.stringProperty);
+        }
+
+        [Test]
+        public void TestDynamicStoreStart()
+        {
+            var cache = TemplateStoreCache();
+
+            Assert.IsNotNull(cache);
+
+            cache.Put(1, cache.Name);
+
+            Assert.AreEqual(cache.Name, CacheTestStore.Map[1]);
+        }
+
+        /// <summary>
+        /// Get's grid name for this test.
+        /// </summary>
+        /// <returns>Grid name.</returns>
+        protected virtual string GridName()
+        {
+            return null;
+        }
+
+        private IDictionary StoreMap()
+        {
+            return CacheTestStore.Map;
+        }
+
+        private ICache<int, string> Cache()
+        {
+            return PortableStoreCache<int, string>();
+        }
+
+        private ICache<TK, TV> PortableStoreCache<TK, TV>()
+        {
+            return Ignition.GetIgnite(GridName()).Cache<TK, TV>(PortableStoreCacheName);
+        }
+
+        private ICache<TK, TV> ObjectStoreCache<TK, TV>()
+        {
+            return Ignition.GetIgnite(GridName()).Cache<TK, TV>(ObjectStoreCacheName);
+        }
+
+        private ICache<int, string> CustomStoreCache()
+        {
+            return Ignition.GetIgnite(GridName()).Cache<int, string>(CustomStoreCacheName);
+        }
+
+        private ICache<int, string> TemplateStoreCache()
+        {
+            var cacheName = TemplateStoreCacheName.Replace("*", Guid.NewGuid().ToString());
+            
+            return Ignition.GetIgnite(GridName()).GetOrCreateCache<int, string>(cacheName);
+        }
+    }
+
+    /// <summary>
+    /// 
+    /// </summary>
+    public class NamedNodeCacheStoreTest : CacheStoreTest
+    {
+        /** <inheritDoc /> */
+        protected override string GridName()
+        {
+            return "name";
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheTestParallelLoadStore.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheTestParallelLoadStore.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheTestParallelLoadStore.cs
new file mode 100644
index 0000000..770ca83
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheTestParallelLoadStore.cs
@@ -0,0 +1,91 @@
+/*
+ * 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.Cache.Store
+{
+    using System.Collections;
+    using System.Collections.Concurrent;
+    using System.Collections.Generic;
+    using System.Linq;
+    using System.Threading;
+    using Apache.Ignite.Core.Cache.Store;
+
+    /// <summary>
+    /// Test cache store with parallel load.
+    /// </summary>
+    public class CacheTestParallelLoadStore : CacheParallelLoadStoreAdapter
+    {
+        /** Length of input data sequence */
+        public const int InputDataLength = 10000;
+
+        /** list of thread ids where Parse has been executed */
+        private static readonly ConcurrentDictionary<int, int> ThreadIds = new ConcurrentDictionary<int, int>();
+
+        /// <summary>
+        /// Gets the count of unique threads that entered Parse method.
+        /// </summary>
+        public static int UniqueThreadCount
+        {
+            get { return ThreadIds.Count; }
+        }
+
+        /// <summary>
+        /// Resets the test counters.
+        /// </summary>
+        public static void ResetCounters()
+        {
+            ThreadIds.Clear();
+        }
+
+        /** <inheritdoc /> */
+        protected override IEnumerable GetInputData()
+        {
+            return Enumerable.Range(0, InputDataLength).Select(x => new Record {Id = x, Name = "Test Record " + x});
+        }
+
+        /** <inheritdoc /> */
+        protected override KeyValuePair<object, object>? Parse(object inputRecord, params object[] args)
+        {
+            var threadId = Thread.CurrentThread.ManagedThreadId;
+            ThreadIds.GetOrAdd(threadId, threadId);
+
+            var minId = (int)args[0];
+
+            var rec = (Record)inputRecord;
+
+            return rec.Id >= minId
+                ? new KeyValuePair<object, object>(rec.Id, rec)
+                : (KeyValuePair<object, object>?) null;
+        }
+
+        /// <summary>
+        /// Test store record.
+        /// </summary>
+        public class Record
+        {
+            /// <summary>
+            /// Gets or sets the identifier.
+            /// </summary>
+            public int Id { get; set; }
+
+            /// <summary>
+            /// Gets or sets the name.
+            /// </summary>
+            public string Name { get; set; }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheTestStore.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheTestStore.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheTestStore.cs
new file mode 100644
index 0000000..9c381cb
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheTestStore.cs
@@ -0,0 +1,155 @@
+/*
+ * 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.Cache.Store
+{
+    using System;
+    using System.Collections;
+    using System.Collections.Concurrent;
+    using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Linq;
+    using System.Threading;
+    using Apache.Ignite.Core.Cache.Store;
+    using Apache.Ignite.Core.Resource;
+
+    [SuppressMessage("ReSharper", "FieldCanBeMadeReadOnly.Local")]
+    public class CacheTestStore : ICacheStore
+    {
+        public static readonly IDictionary Map = new ConcurrentDictionary<object, object>();
+
+        public static bool ExpCommit;
+        
+        public static bool LoadMultithreaded;
+
+        public static bool LoadObjects;
+
+        [InstanceResource]
+        private IIgnite _grid = null;
+
+        [StoreSessionResource]
+#pragma warning disable 649
+        private ICacheStoreSession _ses;
+#pragma warning restore 649
+
+        public static int intProperty;
+
+        public static string stringProperty;
+
+        public static void Reset()
+        {
+            Map.Clear();
+
+            ExpCommit = false;
+            LoadMultithreaded = false;
+            LoadObjects = false;
+        }
+
+        public void LoadCache(Action<object, object> act, params object[] args)
+        {
+            Debug.Assert(_grid != null);
+
+            if (LoadMultithreaded)
+            {
+                int cnt = 0;
+
+                TestUtils.RunMultiThreaded(() => {
+                    int i;
+
+                    while ((i = Interlocked.Increment(ref cnt) - 1) < 1000)
+                        act(i, "val_" + i);
+                }, 8);
+            }
+            else
+            {
+                int start = (int)args[0];
+                int cnt = (int)args[1];
+
+                for (int i = start; i < start + cnt; i++)
+                {
+                    if (LoadObjects)
+                        act(new Key(i), new Value(i));
+                    else
+                        act(i, "val_" + i);
+                }
+            }
+        }
+
+        public object Load(object key)
+        {
+            Debug.Assert(_grid != null);
+
+            return Map[key];
+        }
+
+        public IDictionary LoadAll(ICollection keys)
+        {
+            Debug.Assert(_grid != null);
+
+            return keys.OfType<object>().ToDictionary(key => key, Load);
+        }
+
+        public void Write(object key, object val)
+        {
+            Debug.Assert(_grid != null);
+
+            Map[key] = val;
+        }
+
+        public void WriteAll(IDictionary map)
+        {
+            Debug.Assert(_grid != null);
+
+            foreach (DictionaryEntry e in map)
+                Map[e.Key] = e.Value;
+        }
+
+        public void Delete(object key)
+        {
+            Debug.Assert(_grid != null);
+
+            Map.Remove(key);
+        }
+
+        public void DeleteAll(ICollection keys)
+        {
+            Debug.Assert(_grid != null);
+
+            foreach (object key in keys)
+                Map.Remove(key);
+        }
+
+        public void SessionEnd(bool commit)
+        {
+            Debug.Assert(_grid != null);
+
+            Debug.Assert(_ses != null);
+        }
+
+        public int IntProperty
+        {
+            get { return intProperty; }
+            set { intProperty = value; }
+        }
+
+        public string StringProperty
+        {
+            get { return stringProperty; }
+            set { stringProperty = value; }
+        }
+    }
+}


[40/45] ignite git commit: Platfrom .Net: Fix ignite.common.dll lookup

Posted by ak...@apache.org.
Platfrom .Net: Fix ignite.common.dll lookup

Use Ignite.Common.dll in the dir relative to solution, same way as it is being compiled.


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

Branch: refs/heads/ignite-843
Commit: 2aff2eedade3fe58ecd48af3e8674c884fb796d4
Parents: 5cec202
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Sep 4 17:17:56 2015 +0300
Committer: ptupitsyn <pt...@gridgain.com>
Committed: Fri Sep 4 17:17:56 2015 +0300

----------------------------------------------------------------------
 .../main/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2aff2eed/modules/platform/src/main/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index 52f9b48..6a32729 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -346,22 +346,22 @@
     </ProjectReference>
   </ItemGroup>
   <ItemGroup Condition="'$(Configuration)|$(Platform)' == 'Debug|x64'">
-    <EmbeddedResource Include="..\x64\Debug\ignite.common.dll">
+    <EmbeddedResource Include="$(SolutionDir)\x64\Debug\ignite.common.dll">
       <Link>resources\debug\x64\ignite.common.dll</Link>
     </EmbeddedResource>
   </ItemGroup>
   <ItemGroup Condition="'$(Configuration)|$(Platform)' == 'Release|x64'">
-    <EmbeddedResource Include="..\x64\Release\ignite.common.dll">
+    <EmbeddedResource Include="$(SolutionDir)\x64\Release\ignite.common.dll">
       <Link>resources\release\x64\ignite.common.dll</Link>
     </EmbeddedResource>
   </ItemGroup>
   <ItemGroup Condition="'$(Configuration)|$(Platform)' == 'Debug|x86'">
-    <EmbeddedResource Include="..\Win32\Debug\ignite.common.dll">
+    <EmbeddedResource Include="$(SolutionDir)\Win32\Debug\ignite.common.dll">
       <Link>resources\debug\x86\ignite.common.dll</Link>
     </EmbeddedResource>
   </ItemGroup>
   <ItemGroup Condition="'$(Configuration)|$(Platform)' == 'Release|x86'">
-    <EmbeddedResource Include="..\Win32\Release\ignite.common.dll">
+    <EmbeddedResource Include="$(SolutionDir)\Win32\Release\ignite.common.dll">
       <Link>resources\release\x86\ignite.common.dll</Link>
     </EmbeddedResource>
   </ItemGroup>


[34/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Datastream/IDataStreamer.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Datastream/IDataStreamer.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Datastream/IDataStreamer.cs
new file mode 100644
index 0000000..2713040
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Datastream/IDataStreamer.cs
@@ -0,0 +1,206 @@
+/*
+ * 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.Datastream
+{
+    using System;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Cache.Store;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// Data streamer is responsible for loading external data into cache. It achieves it by
+    /// properly buffering updates and properly mapping keys to nodes responsible for the data
+    /// to make sure that there is the least amount of data movement possible and optimal
+    /// network and memory utilization.
+    /// <para />
+    /// Note that streamer will load data concurrently by multiple internal threads, so the
+    /// data may get to remote nodes in different order from which it was added to
+    /// the streamer.
+    /// <para />
+    /// Also note that <c>IDataStreamer</c> is not the only way to load data into cache.
+    /// Alternatively you can use 
+    /// <see cref="ICacheStore.LoadCache(Action{object, object}, object[])"/>
+    /// method to load data from underlying data store. You can also use standard cache
+    /// <c>put</c> and <c>putAll</c> operations as well, but they most likely will not perform 
+    /// as well as this class for loading data. And finally, data can be loaded from underlying 
+    /// data store on demand, whenever it is accessed - for this no explicit data loading step 
+    /// is needed.
+    /// <para />
+    /// <c>IDataStreamer</c> supports the following configuration properties:
+    /// <list type="bullet">
+    ///     <item>
+    ///         <term>PerNodeBufferSize</term>
+    ///         <description>When entries are added to data streamer they are not sent to Ignite 
+    ///         right away and are buffered internally for better performance and network utilization. 
+    ///         This setting controls the size of internal per-node buffer before buffered data is sent to 
+    ///         remote node. Default value is 1024.</description>
+    ///     </item>
+    ///     <item>
+    ///         <term>PerNodeParallelOperations</term>
+    ///         <description>Sometimes data may be added to the data streamer faster than it can be put 
+    ///         in cache. In this case, new buffered load messages are sent to remote nodes before 
+    ///         responses from previous ones are received. This could cause unlimited heap memory 
+    ///         utilization growth on local and remote nodes. To control memory utilization, this 
+    ///         setting limits maximum allowed number of parallel buffered load messages that are 
+    ///         being processed on remote nodes. If this number is exceeded, then data streamer add/remove
+    ///         methods will block to control memory utilization. Default value is 16.</description>
+    ///     </item>
+    ///     <item>
+    ///         <term>AutoFlushFrequency</term>
+    ///         <description>Automatic flush frequency in milliseconds. Essentially, this is the time 
+    ///         after which the streamer will make an attempt to submit all data added so far to remote 
+    ///         nodes. Note that there is no guarantee that data will be delivered after this concrete 
+    ///         attempt (e.g., it can fail when topology is changing), but it won't be lost anyway. 
+    ///         Disabled by default (default value is <c>0</c>).</description>
+    ///     </item>
+    ///     <item>
+    ///         <term>Isolated</term>
+    ///         <description>Defines if data streamer will assume that there are no other concurrent 
+    ///         updates and allow data streamer choose most optimal concurrent implementation. Default value 
+    ///         is <c>false</c>.</description>
+    ///     </item>
+    /// </list>
+    /// <para/>
+    /// All members are thread-safe and may be used concurrently from multiple threads.
+    /// </summary>
+    public interface IDataStreamer<TK, TV> : IDisposable
+    {
+        /// <summary>
+        /// Name of the cache to load data to.
+        /// </summary>
+        string CacheName { get; }
+
+        /// <summary>
+        /// Flag value indicating that this data streamer assumes that there could be concurrent updates to the cache. 
+        /// <para />
+        /// Default is <code>false</code>.
+        /// </summary>
+        bool AllowOverwrite { get; set; }
+
+        /// <summary>
+        /// Flag indicating that write-through behavior should be disabled for data loading.
+        /// <para />
+        /// Default is <code>false</code>.
+        /// </summary>
+        bool SkipStore { get; set; }
+
+        /// <summary>
+        /// Size of per node key-value pairs buffer.
+        /// <para />
+        /// Setter must be called before any add/remove operation.
+        /// <para />
+        /// Default is <code>1024</code>.
+        /// </summary>
+        int PerNodeBufferSize { get; set; }
+
+        /// <summary>
+        /// Maximum number of parallel load operations for a single node.
+        /// <para />
+        /// Setter must be called before any add/remove operation.
+        /// <para />
+        /// Default is <code>16</code>.
+        /// </summary>
+        int PerNodeParallelOperations { get; set; }
+
+        /// <summary>
+        /// Automatic flush frequency in milliseconds. Essentially, this is the time after which the
+        /// streamer will make an attempt to submit all data added so far to remote nodes.
+        /// Note that there is no guarantee that data will be delivered after this concrete
+        /// attempt (e.g., it can fail when topology is changing), but it won't be lost anyway.
+        /// <para />
+        /// If set to <code>0</code>, automatic flush is disabled.
+        /// <para />
+        /// Default is <code>0</code> (disabled).
+        /// </summary>
+        long AutoFlushFrequency { get; set; }
+
+        /// <summary>
+        /// Gets future for this loading process. This future completes whenever method
+        /// <see cref="IDataStreamer{K,V}.Close(bool)"/> completes.
+        /// </summary>
+        IFuture Future { get; }
+
+        /// <summary>
+        /// Gets or sets custom stream receiver.
+        /// </summary>
+        IStreamReceiver<TK, TV> Receiver { get; set; }
+
+        /// <summary>
+        /// Adds single key-value pair for loading. Passing <c>null</c> as value will be 
+        /// interpreted as removal.
+        /// </summary>
+        /// <param name="key">Key.</param>
+        /// <param name="val">Value.</param>
+        /// <returns>Future for this operation.</returns>
+        IFuture AddData(TK key, TV val);
+
+        /// <summary>
+        /// Adds single key-value pair for loading. Passing <c>null</c> as pair's value will 
+        /// be interpreted as removal.
+        /// </summary>
+        /// <param name="pair">Key-value pair.</param>
+        /// <returns>Future for this operation.</returns>
+        IFuture AddData(KeyValuePair<TK, TV> pair);
+
+        /// <summary>
+        /// Adds collection of key-value pairs for loading. 
+        /// </summary>
+        /// <param name="entries">Entries.</param>
+        /// <returns>Future for this operation.</returns>
+        IFuture AddData(ICollection<KeyValuePair<TK, TV>> entries);
+
+        /// <summary>
+        /// Adds key for removal.
+        /// </summary>
+        /// <param name="key">Key.</param>
+        /// <returns>Future for this operation.</returns>
+        IFuture RemoveData(TK key);
+
+        /// <summary>
+        /// Makes an attempt to load remaining data. This method is mostly similar to 
+        /// <see cref="IDataStreamer{K,V}.Flush()"/> with the difference that it won't wait and 
+        /// will exit immediately.
+        /// </summary>
+        void TryFlush();
+
+        /// <summary>
+        /// Loads any remaining data, but doesn't close the streamer. Data can be still added after
+        /// flush is finished. This method blocks and doesn't allow to add any data until all data
+        /// is loaded.
+        /// </summary>
+        void Flush();
+
+        /// <summary>
+        /// Closes this streamer optionally loading any remaining data.
+        /// </summary>
+        /// <param name="cancel">Whether to cancel ongoing loading operations. When set to <c>true</c>
+        /// there is not guarantees what data will be actually loaded to cache.</param>
+        void Close(bool cancel);
+
+        /// <summary>
+        /// Gets streamer instance with portable mode enabled, changing key and/or value types if necessary.
+        /// In portable mode stream receiver gets data in portable format.
+        /// You can only change key/value types when transitioning from non-portable to portable streamer;
+        /// Changing type of portable streamer is not allowed and will throw an <see cref="InvalidOperationException"/>
+        /// </summary>
+        /// <typeparam name="TK1">Key type in portable mode.</typeparam>
+        /// <typeparam name="TV1">Value type in protable mode.</typeparam>
+        /// <returns>Streamer instance with portable mode enabled.</returns>
+        IDataStreamer<TK1, TV1> WithKeepPortable<TK1, TV1>();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Datastream/IStreamReceiver.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Datastream/IStreamReceiver.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Datastream/IStreamReceiver.cs
new file mode 100644
index 0000000..d75dc54
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Datastream/IStreamReceiver.cs
@@ -0,0 +1,38 @@
+/*
+ * 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.Datastream
+{
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Cache;
+
+    /// <summary>
+    /// Updates cache with batch of entries. 
+    /// Usually it is enough to configure <see cref="IDataStreamer{K,V}.AllowOverwrite" /> property and appropriate 
+    /// internal cache receiver will be chosen automatically. But in some cases custom implementation may help 
+    /// to achieve better performance.
+    /// </summary>
+    public interface IStreamReceiver<TK, TV>
+    {
+        /// <summary>
+        /// Updates cache with batch of entries.
+        /// </summary>
+        /// <param name="cache">Cache.</param>
+        /// <param name="entries">Entries.</param>
+        void Receive(ICache<TK, TV> cache, ICollection<ICacheEntry<TK, TV>> entries);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Datastream/StreamTransformer.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Datastream/StreamTransformer.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Datastream/StreamTransformer.cs
new file mode 100644
index 0000000..0398342
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Datastream/StreamTransformer.cs
@@ -0,0 +1,73 @@
+/*
+ * 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.Datastream
+{
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Datastream;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Convenience adapter to transform update existing values in streaming cache 
+    /// based on the previously cached value.
+    /// </summary>
+    /// <typeparam name="TK">Key type.</typeparam>
+    /// <typeparam name="TV">Value type.</typeparam>
+    /// <typeparam name="TA">The type of the processor argument.</typeparam>
+    /// <typeparam name="TR">The type of the processor result.</typeparam>
+    public sealed class StreamTransformer<TK, TV, TA, TR> : IStreamReceiver<TK, TV>, 
+        IPortableWriteAware
+    {
+        /** Entry processor. */
+        private readonly ICacheEntryProcessor<TK, TV, TA, TR> _proc;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="StreamTransformer{K, V, A, R}"/> class.
+        /// </summary>
+        /// <param name="proc">Entry processor.</param>
+        public StreamTransformer(ICacheEntryProcessor<TK, TV, TA, TR> proc)
+        {
+            IgniteArgumentCheck.NotNull(proc, "proc");
+
+            _proc = proc;
+        }
+
+        /** <inheritdoc /> */
+        public void Receive(ICache<TK, TV> cache, ICollection<ICacheEntry<TK, TV>> entries)
+        {
+            var keys = new List<TK>(entries.Count);
+
+            foreach (var entry in entries)
+                keys.Add(entry.Key);
+
+            cache.InvokeAll(keys, _proc, default(TA));
+        }
+
+        /** <inheritdoc /> */
+        void IPortableWriteAware.WritePortable(IPortableWriter writer)
+        {
+            var w = (PortableWriterImpl)writer;
+
+            w.WriteByte(StreamReceiverHolder.RcvTransformer);
+
+            PortableUtils.WritePortableOrSerializable(w, _proc);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Datastream/StreamVisitor.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Datastream/StreamVisitor.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Datastream/StreamVisitor.cs
new file mode 100644
index 0000000..5d155d7
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Datastream/StreamVisitor.cs
@@ -0,0 +1,55 @@
+/*
+ * 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.Datastream
+{
+    using System;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Impl.Common;
+
+    /// <summary>
+    /// Convenience adapter to visit every key-value tuple in the stream.
+    /// Note that the visitor does not update the cache.
+    /// </summary>
+    /// <typeparam name="TK">The type of the cache key.</typeparam>
+    /// <typeparam name="TV">The type of the cache value.</typeparam>
+    [Serializable]
+    public sealed class StreamVisitor<TK, TV> : IStreamReceiver<TK, TV>
+    {
+        /** Visitor action */
+        private readonly Action<ICache<TK, TV>, ICacheEntry<TK, TV>> _action;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="StreamVisitor{K, V}"/> class.
+        /// </summary>
+        /// <param name="action">The action to be called on each stream entry.</param>
+        public StreamVisitor(Action<ICache<TK, TV>, ICacheEntry<TK, TV>> action)
+        {
+            IgniteArgumentCheck.NotNull(action, "action");
+
+            _action = action;
+        }
+
+        /** <inheritdoc /> */
+        public void Receive(ICache<TK, TV> cache, ICollection<ICacheEntry<TK, TV>> entries)
+        {
+            foreach (var entry in entries)
+                _action(cache, entry);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/CacheEvent.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/CacheEvent.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/CacheEvent.cs
new file mode 100644
index 0000000..ff5084b
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/CacheEvent.cs
@@ -0,0 +1,176 @@
+/*
+ * 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.Events
+{
+    using System;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// In-memory database (cache) event.
+    /// </summary>
+    public sealed class CacheEvent : EventBase
+	{
+        /** */
+        private readonly string _cacheName;
+
+        /** */
+        private readonly int _partition;
+
+        /** */
+        private readonly bool _isNear;
+
+        /** */
+        private readonly IClusterNode _eventNode;
+
+        /** */
+        private readonly object _key;
+
+        /** */
+        private readonly IgniteGuid _xid;
+
+        /** */
+        private readonly object _lockId;
+
+        /** */
+        private readonly object _newValue;
+
+        /** */
+        private readonly object _oldValue;
+
+        /** */
+        private readonly bool _hasOldValue;
+
+        /** */
+        private readonly bool _hasNewValue;
+
+        /** */
+        private readonly Guid _subjectId;
+
+        /** */
+        private readonly string _closureClassName;
+
+        /** */
+        private readonly string _taskName;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="r">The reader to read data from.</param>
+        internal CacheEvent(IPortableRawReader r) : base(r)
+        {
+            _cacheName = r.ReadString();
+            _partition = r.ReadInt();
+            _isNear = r.ReadBoolean();
+            _eventNode = ReadNode(r);
+            _key = r.ReadObject<object>();
+            _xid = IgniteGuid.ReadPortable(r);
+            _lockId = r.ReadObject<object>();
+            _newValue = r.ReadObject<object>();
+            _oldValue = r.ReadObject<object>();
+            _hasOldValue = r.ReadBoolean();
+            _hasNewValue = r.ReadBoolean();
+            _subjectId = r.ReadGuid() ?? Guid.Empty;
+            _closureClassName = r.ReadString();
+            _taskName = r.ReadString();
+        }
+		
+        /// <summary>
+        /// Gets cache name. 
+        /// </summary>
+        public string CacheName { get { return _cacheName; } }
+
+        /// <summary>
+        /// Gets partition for the event which is the partition the key belongs to. 
+        /// </summary>
+        public int Partition { get { return _partition; } }
+
+        /// <summary>
+        /// Gets flag indicating whether event happened on near or partitioned cache. 
+        /// </summary>
+        public bool IsNear { get { return _isNear; } }
+
+        /// <summary>
+        /// Gets node which initiated cache operation or null if that node is not available. 
+        /// </summary>
+        public IClusterNode EventNode { get { return _eventNode; } }
+
+        /// <summary>
+        /// Gets cache entry associated with event. 
+        /// </summary>
+        public object Key { get { return _key; } }
+
+        /// <summary>
+        /// ID of surrounding cache cache transaction or null if there is no surrounding transaction. 
+        /// </summary>
+        public IgniteGuid Xid { get { return _xid; } }
+
+        /// <summary>
+        /// ID of the lock if held or null if no lock held. 
+        /// </summary>
+        public object LockId { get { return _lockId; } }
+
+        /// <summary>
+        /// Gets new value for this event. 
+        /// </summary>
+        public object NewValue { get { return _newValue; } }
+
+        /// <summary>
+        /// Gets old value associated with this event. 
+        /// </summary>
+        public object OldValue { get { return _oldValue; } }
+
+        /// <summary>
+        /// Gets flag indicating whether cache entry has old value in case if we only have old value in serialized form 
+        /// in which case <see cref="OldValue" /> will return null. 
+        /// </summary>
+        public bool HasOldValue { get { return _hasOldValue; } }
+
+        /// <summary>
+        /// Gets flag indicating whether cache entry has new value in case if we only have new value in serialized form 
+        /// in which case <see cref="NewValue" /> will return null. 
+        /// </summary>
+        public bool HasNewValue { get { return _hasNewValue; } }
+
+        /// <summary>
+        /// Gets security subject ID initiated this cache event, if available. This property is available only for <see 
+        /// cref="EventType.EvtCacheObjectPut" />, <see cref="EventType.EvtCacheObjectRemoved" /> and <see 
+        /// cref="EventType.EvtCacheObjectRead" /> cache events. Subject ID will be set either to nodeId initiated 
+        /// cache update or read or client ID initiated cache update or read. 
+        /// </summary>
+        public Guid SubjectId { get { return _subjectId; } }
+
+        /// <summary>
+        /// Gets closure class name (applicable only for TRANSFORM operations). 
+        /// </summary>
+        public string ClosureClassName { get { return _closureClassName; } }
+
+        /// <summary>
+        /// Gets task name if cache event was caused by an operation initiated within task execution. 
+        /// </summary>
+        public string TaskName { get { return _taskName; } }
+        
+        /** <inheritDoc /> */
+	    public override string ToShortString()
+	    {
+	        return string.Format("{0}: IsNear={1}, Key={2}, HasNewValue={3}, HasOldValue={4}, NodeId={5}", Name, 
+                _isNear, _key, HasNewValue, HasOldValue, Node.Id);
+	    }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/CacheQueryExecutedEvent.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/CacheQueryExecutedEvent.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/CacheQueryExecutedEvent.cs
new file mode 100644
index 0000000..8443c68
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/CacheQueryExecutedEvent.cs
@@ -0,0 +1,97 @@
+/*
+ * 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.Events
+{
+    using System;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Cache query execution event.
+    /// </summary>
+    public sealed class CacheQueryExecutedEvent : EventBase
+	{
+        /** */
+        private readonly string _queryType;
+
+        /** */
+        private readonly string _cacheName;
+
+        /** */
+        private readonly string _className;
+
+        /** */
+        private readonly string _clause;
+
+        /** */
+        private readonly Guid _subjectId;
+
+        /** */
+        private readonly string _taskName;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="r">The reader to read data from.</param>
+        internal CacheQueryExecutedEvent(IPortableRawReader r) : base(r)
+        {
+            _queryType = r.ReadString();
+            _cacheName = r.ReadString();
+            _className = r.ReadString();
+            _clause = r.ReadString();
+            _subjectId = r.ReadGuid() ?? Guid.Empty;
+            _taskName = r.ReadString();
+        }
+		
+        /// <summary>
+        /// Gets query type. 
+        /// </summary>
+        public string QueryType { get { return _queryType; } }
+
+        /// <summary>
+        /// Gets cache name on which query was executed. 
+        /// </summary>
+        public string CacheName { get { return _cacheName; } }
+
+        /// <summary>
+        /// Gets queried class name. Applicable for SQL and full text queries. 
+        /// </summary>
+        public string ClassName { get { return _className; } }
+
+        /// <summary>
+        /// Gets query clause. Applicable for SQL, SQL fields and full text queries. 
+        /// </summary>
+        public string Clause { get { return _clause; } }
+
+        /// <summary>
+        /// Gets security subject ID. 
+        /// </summary>
+        public Guid SubjectId { get { return _subjectId; } }
+
+        /// <summary>
+        /// Gets the name of the task that executed the query (if any). 
+        /// </summary>
+        public string TaskName { get { return _taskName; } }
+
+        /** <inheritDoc /> */
+	    public override string ToShortString()
+	    {
+	        return string.Format("{0}: QueryType={1}, CacheName={2}, ClassName={3}, Clause={4}, SubjectId={5}, " +
+	                             "TaskName={6}", Name, QueryType, CacheName, ClassName, Clause, SubjectId, TaskName);
+	    }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/CacheQueryReadEvent.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/CacheQueryReadEvent.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/CacheQueryReadEvent.cs
new file mode 100644
index 0000000..7338eab
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/CacheQueryReadEvent.cs
@@ -0,0 +1,134 @@
+/*
+ * 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.Events
+{
+    using System;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Cache query read event.
+    /// </summary>
+    public sealed class CacheQueryReadEvent : EventBase
+	{
+        /** */
+        private readonly string _queryType;
+
+        /** */
+        private readonly string _cacheName;
+
+        /** */
+        private readonly string _className;
+
+        /** */
+        private readonly string _clause;
+
+        /** */
+        private readonly Guid _subjectId;
+
+        /** */
+        private readonly string _taskName;
+
+        /** */
+        private readonly object _key;
+
+        /** */
+        private readonly object _value;
+
+        /** */
+        private readonly object _oldValue;
+
+        /** */
+        private readonly object _row;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="r">The reader to read data from.</param>
+        internal CacheQueryReadEvent(IPortableRawReader r) : base(r)
+        {
+            _queryType = r.ReadString();
+            _cacheName = r.ReadString();
+            _className = r.ReadString();
+            _clause = r.ReadString();
+            _subjectId = r.ReadGuid() ?? Guid.Empty;
+            _taskName = r.ReadString();
+            _key = r.ReadObject<object>();
+            _value = r.ReadObject<object>();
+            _oldValue = r.ReadObject<object>();
+            _row = r.ReadObject<object>();
+        }
+		
+        /// <summary>
+        /// Gets query type. 
+        /// </summary>
+        public string QueryType { get { return _queryType; } }
+
+        /// <summary>
+        /// Gets cache name on which query was executed. 
+        /// </summary>
+        public string CacheName { get { return _cacheName; } }
+
+        /// <summary>
+        /// Gets queried class name. Applicable for SQL and full text queries. 
+        /// </summary>
+        public string ClassName { get { return _className; } }
+
+        /// <summary>
+        /// Gets query clause. Applicable for SQL, SQL fields and full text queries. 
+        /// </summary>
+        public string Clause { get { return _clause; } }
+
+        /// <summary>
+        /// Gets security subject ID. 
+        /// </summary>
+        public Guid SubjectId { get { return _subjectId; } }
+
+        /// <summary>
+        /// Gets the name of the task that executed the query (if any). 
+        /// </summary>
+        public string TaskName { get { return _taskName; } }
+
+        /// <summary>
+        /// Gets read entry key. 
+        /// </summary>
+        public object Key { get { return _key; } }
+
+        /// <summary>
+        /// Gets read entry value. 
+        /// </summary>
+        public object Value { get { return _value; } }
+
+        /// <summary>
+        /// Gets read entry old value (applicable for continuous queries). 
+        /// </summary>
+        public object OldValue { get { return _oldValue; } }
+
+        /// <summary>
+        /// Gets read results set row. 
+        /// </summary>
+        public object Row { get { return _row; } }
+
+        /** <inheritDoc /> */
+	    public override string ToShortString()
+	    {
+	        return string.Format("{0}: QueryType={1}, CacheName={2}, ClassName={3}, Clause={4}, SubjectId={5}, " +
+	                             "TaskName={6}, Key={7}, Value={8}, OldValue={9}, Row={10}", Name, QueryType, 
+                                 CacheName, ClassName, Clause, SubjectId, TaskName, Key, Value, OldValue, Row);
+	    }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/CacheRebalancingEvent.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/CacheRebalancingEvent.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/CacheRebalancingEvent.cs
new file mode 100644
index 0000000..656550a
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/CacheRebalancingEvent.cs
@@ -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.
+ */
+
+namespace Apache.Ignite.Core.Events
+{
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// In-memory database (cache) rebalancing event. Rebalance event happens every time there is a change
+    /// </summary>
+    public sealed class CacheRebalancingEvent : EventBase
+	{
+        /** */
+        private readonly string _cacheName;
+
+        /** */
+        private readonly int _partition;
+
+        /** */
+        private readonly IClusterNode _discoveryNode;
+
+        /** */
+        private readonly int _discoveryEventType;
+
+        /** */
+        private readonly string _discoveryEventName;
+
+        /** */
+        private readonly long _discoveryTimestamp;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="r">The reader to read data from.</param>
+        internal CacheRebalancingEvent(IPortableRawReader r) : base(r)
+        {
+            _cacheName = r.ReadString();
+            _partition = r.ReadInt();
+            _discoveryNode = ReadNode(r);
+            _discoveryEventType = r.ReadInt();
+            _discoveryEventName = r.ReadString();
+            _discoveryTimestamp = r.ReadLong();
+        }
+		
+        /// <summary>
+        /// Gets cache name. 
+        /// </summary>
+        public string CacheName { get { return _cacheName; } }
+
+        /// <summary>
+        /// Gets partition for the event. 
+        /// </summary>
+        public int Partition { get { return _partition; } }
+
+        /// <summary>
+        /// Gets shadow of the node that triggered this rebalancing event. 
+        /// </summary>
+        public IClusterNode DiscoveryNode { get { return _discoveryNode; } }
+
+        /// <summary>
+        /// Gets type of discovery event that triggered this rebalancing event. 
+        /// </summary>
+        public int DiscoveryEventType { get { return _discoveryEventType; } }
+
+        /// <summary>
+        /// Gets name of discovery event that triggered this rebalancing event. 
+        /// </summary>
+        public string DiscoveryEventName { get { return _discoveryEventName; } }
+
+        /// <summary>
+        /// Gets timestamp of discovery event that caused this rebalancing event. 
+        /// </summary>
+        public long DiscoveryTimestamp { get { return _discoveryTimestamp; } }
+
+        /** <inheritDoc /> */
+	    public override string ToShortString()
+	    {
+	        return string.Format("{0}: CacheName={1}, Partition={2}, DiscoveryNode={3}, DiscoveryEventType={4}, " +
+	                             "DiscoveryEventName={5}, DiscoveryTimestamp={6}", Name, CacheName, Partition,
+	                             DiscoveryNode, DiscoveryEventType, DiscoveryEventName, DiscoveryTimestamp);
+	    }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/CheckpointEvent.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/CheckpointEvent.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/CheckpointEvent.cs
new file mode 100644
index 0000000..7b7ea59
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/CheckpointEvent.cs
@@ -0,0 +1,50 @@
+/*
+ * 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.Events
+{
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Grid checkpoint event.
+    /// </summary>
+    public sealed class CheckpointEvent : EventBase
+	{
+        /** */
+        private readonly string _key;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="r">The reader to read data from.</param>
+        internal CheckpointEvent(IPortableRawReader r) : base(r)
+        {
+            _key = r.ReadString();
+        }
+		
+        /// <summary>
+        /// Gets checkpoint key associated with this event. 
+        /// </summary>
+        public string Key { get { return _key; } }
+
+        /** <inheritDoc /> */
+	    public override string ToShortString()
+	    {
+	        return string.Format("{0}: Key={1}", Name, Key);
+	    }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/DiscoveryEvent.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/DiscoveryEvent.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/DiscoveryEvent.cs
new file mode 100644
index 0000000..5b5443c
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/DiscoveryEvent.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.Events
+{
+    using System.Collections.Generic;
+    using System.Collections.ObjectModel;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Grid discovery event.
+    /// </summary>
+    public sealed class DiscoveryEvent : EventBase
+	{
+        /** */
+        private readonly IClusterNode _eventNode;
+
+        /** */
+        private readonly long _topologyVersion;
+
+        /** */
+        private readonly ReadOnlyCollection<IClusterNode> _topologyNodes;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="r">The reader to read data from.</param>
+        internal DiscoveryEvent(IPortableRawReader r) : base(r)
+        {
+            _eventNode = ReadNode(r);
+            _topologyVersion = r.ReadLong();
+
+            var nodes = IgniteUtils.ReadNodes(r);
+
+            _topologyNodes = nodes == null ? null : new ReadOnlyCollection<IClusterNode>(nodes);
+        }
+
+        /// <summary>
+        /// Gets node that caused this event to be generated. It is potentially different from the node on which this 
+        /// event was recorded. For example, node A locally recorded the event that a remote node B joined the topology. 
+        /// In this case this method will return ID of B. 
+        /// </summary>
+        public IClusterNode EventNode { get { return _eventNode; } }
+
+        /// <summary>
+        /// Gets topology version if this event is raised on topology change and configured discovery
+        /// SPI implementation supports topology versioning.
+        /// </summary>
+        public long TopologyVersion { get { return _topologyVersion; } }
+
+        /// <summary>
+        /// Gets topology nodes from topology snapshot. If SPI implementation does not support versioning, the best 
+        /// effort snapshot will be captured. 
+        /// </summary>
+        public ICollection<IClusterNode> TopologyNodes { get { return _topologyNodes; } }
+
+        /** <inheritDoc /> */
+	    public override string ToShortString()
+	    {
+	        return string.Format("{0}: EventNode={1}, TopologyVersion={2}, TopologyNodes={3}", Name, EventNode, 
+                TopologyVersion, TopologyNodes.Count);
+	    }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/EventBase.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/EventBase.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/EventBase.cs
new file mode 100644
index 0000000..2b905a1
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/EventBase.cs
@@ -0,0 +1,160 @@
+/*
+ * 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.Events
+{
+    using System;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Base event implementation.
+    /// </summary>
+    public abstract class EventBase : IEvent, IEquatable<EventBase>
+    {
+        /** */
+        private readonly IgniteGuid _id;
+
+        /** */
+        private readonly long _localOrder;
+
+        /** */
+        private readonly IClusterNode _node;
+
+        /** */
+        private readonly string _message;
+
+        /** */
+        private readonly int _type;
+
+        /** */
+        private readonly string _name;
+
+        /** */
+        private readonly DateTime _timeStamp;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="EventBase"/> class.
+        /// </summary>
+        /// <param name="r">The reader to read data from.</param>
+        protected EventBase(IPortableRawReader r)
+        {
+            _id = IgniteGuid.ReadPortable(r);
+
+            _localOrder = r.ReadLong();
+
+            _node = ReadNode(r);
+
+            _message = r.ReadString();
+            _type = r.ReadInt();
+            _name = r.ReadString();
+            _timeStamp = r.ReadDate() ?? DateTime.Now;
+        }
+
+        /** <inheritDoc /> */
+        public IgniteGuid Id
+        {
+            get { return _id; }
+        }
+
+        /** <inheritDoc /> */
+        public long LocalOrder
+        {
+            get { return _localOrder; }
+        }
+
+        /** <inheritDoc /> */
+        public IClusterNode Node
+        {
+            get { return _node; }
+        }
+
+        /** <inheritDoc /> */
+        public string Message
+        {
+            get { return _message; }
+        }
+
+        /** <inheritDoc /> */
+        public int Type
+        {
+            get { return _type; }
+        }
+
+        /** <inheritDoc /> */
+        public string Name
+        {
+            get { return _name; }
+        }
+
+        /** <inheritDoc /> */
+        public DateTime TimeStamp
+        {
+            get { return _timeStamp; }
+        }
+
+        /** <inheritDoc /> */
+        public virtual string ToShortString()
+        {
+            return ToString();
+        }
+
+        /** <inheritDoc /> */
+        public bool Equals(EventBase other)
+        {
+            if (ReferenceEquals(null, other)) return false;
+            if (ReferenceEquals(this, other)) return true;
+            
+            return _id.Equals(other._id);
+        }
+
+        /** <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((EventBase) obj);
+        }
+
+        /** <inheritDoc /> */
+        public override int GetHashCode()
+        {
+            return _id.GetHashCode();
+        }
+
+        /** <inheritDoc /> */
+        public override string ToString()
+        {
+            return string.Format("CacheEntry [Name={0}, Type={1}, TimeStamp={2}, Message={3}]", Name, Type, TimeStamp,
+                Message);
+        }
+
+        /// <summary>
+        /// Reads a node from stream.
+        /// </summary>
+        /// <param name="reader">Reader.</param>
+        /// <returns>Node or null.</returns>
+        protected static IClusterNode ReadNode(IPortableRawReader reader)
+        {
+            return ((PortableReaderImpl)reader).Marshaller.Ignite.GetNode(reader.ReadGuid());
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/EventReader.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/EventReader.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/EventReader.cs
new file mode 100644
index 0000000..aa9f538
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/EventReader.cs
@@ -0,0 +1,72 @@
+/*
+ * 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.Events
+{
+    using System;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Event reader.
+    /// </summary>
+    internal static class EventReader
+    {
+        /// <summary>
+        /// Reads an event.
+        /// </summary>
+        /// <typeparam name="T">Type of the event</typeparam>
+        /// <param name="reader">Reader.</param>
+        /// <returns>Deserialized event.</returns>
+        /// <exception cref="System.InvalidCastException">Incompatible event type.</exception>
+        public static T Read<T>(IPortableReader reader) where T : IEvent
+        {
+            var r = reader.RawReader();
+
+            var clsId = r.ReadInt();
+
+            if (clsId == -1)
+                return default(T);
+
+            return (T) CreateInstance(clsId, r);
+        }
+
+        /// <summary>
+        /// Creates an event instance by type id.
+        /// </summary>
+        /// <param name="clsId">Type id.</param>
+        /// <param name="reader">Reader.</param>
+        /// <returns>Created and deserialized instance.</returns>
+        /// <exception cref="System.InvalidOperationException">Invalid event class id:  + clsId</exception>
+        private static IEvent CreateInstance(int clsId, IPortableRawReader reader)
+        {
+            switch (clsId)
+            {
+                case 2: return new CacheEvent(reader);
+                case 3: return new CacheQueryExecutedEvent(reader);
+                case 4: return new CacheQueryReadEvent(reader);
+                case 5: return new CacheRebalancingEvent(reader);
+                case 6: return new CheckpointEvent(reader);
+                case 7: return new DiscoveryEvent(reader);
+                case 8: return new JobEvent(reader);
+                case 9: return new SwapSpaceEvent(reader);
+                case 10: return new TaskEvent(reader);
+            }
+
+            throw new InvalidOperationException("Invalid event class id: " + clsId);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/EventType.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/EventType.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/EventType.cs
new file mode 100644
index 0000000..1e649bb
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/EventType.cs
@@ -0,0 +1,514 @@
+/*
+ * 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.Events
+{
+    using System.Diagnostics.CodeAnalysis;
+    using System.Linq;
+    using System.Reflection;
+
+    /// <summary>
+    /// Contains event type constants. The decision to use class and not enumeration is dictated 
+    /// by allowing users to create their own events and/or event types which would be impossible with enumerations.
+    /// <para />
+    /// Note that this interface defines not only individual type constants, 
+    /// but arrays of types as well to be conveniently used with <see cref="IEvents"/> methods.
+    /// <para />
+    /// NOTE: all types in range <b>from 1 to 1000 are reserved</b> for internal Ignite events 
+    /// and should not be used by user-defined events.
+    /// </summary>
+    public static class EventType
+    {
+        /// <summary>
+        /// Built-in event type: checkpoint was saved.
+        /// </summary>
+        public static readonly int EvtCheckpointSaved = 1;
+
+        /// <summary>
+        /// Built-in event type: checkpoint was loaded.
+        /// </summary>
+        public static readonly int EvtCheckpointLoaded = 2;
+
+        /// <summary>
+        /// Built-in event type: checkpoint was removed. Reasons are: timeout expired, or or it was manually removed, 
+        /// or it was automatically removed by the task session.
+        /// </summary>
+        public static readonly int EvtCheckpointRemoved = 3;
+
+        /// <summary>
+        /// Built-in event type: node joined topology. New node has been discovered and joined grid topology. Note that 
+        /// even though a node has been discovered there could be a number of warnings in the log. In certain 
+        /// situations Ignite doesn't prevent a node from joining but prints warning messages into the log.
+        /// </summary>
+        public static readonly int EvtNodeJoined = 10;
+
+        /// <summary>
+        /// Built-in event type: node has normally left topology.
+        /// </summary>
+        public static readonly int EvtNodeLeft = 11;
+
+        /// <summary>
+        /// Built-in event type: node failed. Ignite detected that node has presumably crashed and is considered 
+        /// failed.
+        /// </summary>
+        public static readonly int EvtNodeFailed = 12;
+
+        /// <summary>
+        /// Built-in event type: node metrics updated. Generated when node's metrics are updated. In most cases this 
+        /// callback is invoked with every heartbeat received from a node (including local node).
+        /// </summary>
+        public static readonly int EvtNodeMetricsUpdated = 13;
+
+        /// <summary>
+        /// Built-in event type: local node segmented. Generated when node determines that it runs in invalid network 
+        /// segment.
+        /// </summary>
+        public static readonly int EvtNodeSegmented = 14;
+
+        /// <summary>
+        /// Built-in event type: client node disconnected.
+        /// </summary>
+        public static readonly int EvtClientNodeDisconnected = 16;
+
+        /// <summary>
+        /// Built-in event type: client node reconnected.
+        /// </summary>
+        public static readonly int EvtClientNodeReconnected = 17;
+
+        /// <summary>
+        /// Built-in event type: task started.
+        /// </summary>
+        public static readonly int EvtTaskStarted = 20;
+
+        /// <summary>
+        /// Built-in event type: task finished. Task got finished. This event is triggered every time a task finished 
+        /// without exception.
+        /// </summary>
+        public static readonly int EvtTaskFinished = 21;
+
+        /// <summary>
+        /// Built-in event type: task failed. Task failed. This event is triggered every time a task finished with an 
+        /// exception. Note that prior to this event, there could be other events recorded specific to the failure.
+        /// </summary>
+        public static readonly int EvtTaskFailed = 22;
+
+        /// <summary>
+        /// Built-in event type: task timed out.
+        /// </summary>
+        public static readonly int EvtTaskTimedout = 23;
+
+        /// <summary>
+        /// Built-in event type: task session attribute set.
+        /// </summary>
+        public static readonly int EvtTaskSessionAttrSet = 24;
+
+        /// <summary>
+        /// Built-in event type: task reduced.
+        /// </summary>
+        public static readonly int EvtTaskReduced = 25;
+
+        /// <summary>
+        /// Built-in event type: Ignite job was mapped in {@link org.apache.ignite.compute.ComputeTask#map(List, Object)} 
+        /// method.
+        /// </summary>
+        public static readonly int EvtJobMapped = 40;
+
+        /// <summary>
+        /// Built-in event type: Ignite job result was received by {@link 
+        /// org.apache.ignite.compute.ComputeTask#result(org.apache.ignite.compute.ComputeJobResult, List)} method.
+        /// </summary>
+        public static readonly int EvtJobResulted = 41;
+
+        /// <summary>
+        /// Built-in event type: Ignite job failed over.
+        /// </summary>
+        public static readonly int EvtJobFailedOver = 43;
+
+        /// <summary>
+        /// Built-in event type: Ignite job started.
+        /// </summary>
+        public static readonly int EvtJobStarted = 44;
+
+        /// <summary>
+        /// Built-in event type: Ignite job finished. Job has successfully completed and produced a result which from the 
+        /// user perspective can still be either negative or positive.
+        /// </summary>
+        public static readonly int EvtJobFinished = 45;
+
+        /// <summary>
+        /// Built-in event type: Ignite job timed out.
+        /// </summary>
+        public static readonly int EvtJobTimedout = 46;
+
+        /// <summary>
+        /// Built-in event type: Ignite job rejected during collision resolution.
+        /// </summary>
+        public static readonly int EvtJobRejected = 47;
+
+        /// <summary>
+        /// Built-in event type: Ignite job failed. Job has failed. This means that there was some error event during job 
+        /// execution and job did not produce a result.
+        /// </summary>
+        public static readonly int EvtJobFailed = 48;
+
+        /// <summary>
+        /// Built-in event type: Ignite job queued. Job arrived for execution and has been queued (added to passive queue 
+        /// during collision resolution).
+        /// </summary>
+        public static readonly int EvtJobQueued = 49;
+
+        /// <summary>
+        /// Built-in event type: Ignite job cancelled.
+        /// </summary>
+        public static readonly int EvtJobCancelled = 50;
+
+        /// <summary>
+        /// Built-in event type: entry created.
+        /// </summary>
+        public static readonly int EvtCacheEntryCreated = 60;
+
+        /// <summary>
+        /// Built-in event type: entry destroyed.
+        /// </summary>
+        public static readonly int EvtCacheEntryDestroyed = 61;
+
+        /// <summary>
+        /// Built-in event type: entry evicted.
+        /// </summary>
+        public static readonly int EvtCacheEntryEvicted = 62;
+
+        /// <summary>
+        /// Built-in event type: object put.
+        /// </summary>
+        public static readonly int EvtCacheObjectPut = 63;
+
+        /// <summary>
+        /// Built-in event type: object read.
+        /// </summary>
+        public static readonly int EvtCacheObjectRead = 64;
+
+        /// <summary>
+        /// Built-in event type: object removed.
+        /// </summary>
+        public static readonly int EvtCacheObjectRemoved = 65;
+
+        /// <summary>
+        /// Built-in event type: object locked.
+        /// </summary>
+        public static readonly int EvtCacheObjectLocked = 66;
+
+        /// <summary>
+        /// Built-in event type: object unlocked.
+        /// </summary>
+        public static readonly int EvtCacheObjectUnlocked = 67;
+
+        /// <summary>
+        /// Built-in event type: cache object swapped from swap storage.
+        /// </summary>
+        public static readonly int EvtCacheObjectSwapped = 68;
+
+        /// <summary>
+        /// Built-in event type: cache object unswapped from swap storage.
+        /// </summary>
+        public static readonly int EvtCacheObjectUnswapped = 69;
+
+        /// <summary>
+        /// Built-in event type: cache object was expired when reading it.
+        /// </summary>
+        public static readonly int EvtCacheObjectExpired = 70;
+
+        /// <summary>
+        /// Built-in event type: swap space data read.
+        /// </summary>
+        public static readonly int EvtSwapSpaceDataRead = 71;
+
+        /// <summary>
+        /// Built-in event type: swap space data stored.
+        /// </summary>
+        public static readonly int EvtSwapSpaceDataStored = 72;
+
+        /// <summary>
+        /// Built-in event type: swap space data removed.
+        /// </summary>
+        public static readonly int EvtSwapSpaceDataRemoved = 73;
+
+        /// <summary>
+        /// Built-in event type: swap space cleared.
+        /// </summary>
+        public static readonly int EvtSwapSpaceCleared = 74;
+
+        /// <summary>
+        /// Built-in event type: swap space data evicted.
+        /// </summary>
+        public static readonly int EvtSwapSpaceDataEvicted = 75;
+
+        /// <summary>
+        /// Built-in event type: cache object stored in off-heap storage.
+        /// </summary>
+        public static readonly int EvtCacheObjectToOffheap = 76;
+
+        /// <summary>
+        /// Built-in event type: cache object moved from off-heap storage back into memory.
+        /// </summary>
+        public static readonly int EvtCacheObjectFromOffheap = 77;
+
+        /// <summary>
+        /// Built-in event type: cache rebalance started.
+        /// </summary>
+        public static readonly int EvtCacheRebalanceStarted = 80;
+
+        /// <summary>
+        /// Built-in event type: cache rebalance stopped.
+        /// </summary>
+        public static readonly int EvtCacheRebalanceStopped = 81;
+
+        /// <summary>
+        /// Built-in event type: cache partition loaded.
+        /// </summary>
+        public static readonly int EvtCacheRebalancePartLoaded = 82;
+
+        /// <summary>
+        /// Built-in event type: cache partition unloaded.
+        /// </summary>
+        public static readonly int EvtCacheRebalancePartUnloaded = 83;
+
+        /// <summary>
+        /// Built-in event type: cache entry rebalanced.
+        /// </summary>
+        public static readonly int EvtCacheRebalanceObjectLoaded = 84;
+
+        /// <summary>
+        /// Built-in event type: cache entry unloaded.
+        /// </summary>
+        public static readonly int EvtCacheRebalanceObjectUnloaded = 85;
+
+        /// <summary>
+        /// Built-in event type: all nodes that hold partition left topology.
+        /// </summary>
+        public static readonly int EvtCacheRebalancePartDataLost = 86;
+
+        /// <summary>
+        /// Built-in event type: query executed.
+        /// </summary>
+        public static readonly int EvtCacheQueryExecuted = 96;
+
+        /// <summary>
+        /// Built-in event type: query entry read.
+        /// </summary>
+        public static readonly int EvtCacheQueryObjectRead = 97;
+
+        /// <summary>
+        /// Built-in event type: cache started.
+        /// </summary>
+        public static readonly int EvtCacheStarted = 98;
+
+        /// <summary>
+        /// Built-in event type: cache started.
+        /// </summary>
+        public static readonly int EvtCacheStopped = 99;
+
+        /// <summary>
+        /// Built-in event type: cache nodes left.
+        /// </summary>
+        public static readonly int EvtCacheNodesLeft = 100;
+
+        /// <summary>
+        /// All events indicating an error or failure condition. It is convenient to use when fetching all events 
+        /// indicating error or failure.
+        /// </summary>
+        [SuppressMessage("Microsoft.Security", "CA2105:ArrayFieldsShouldNotBeReadOnly",
+            Justification = "Breaking change. Should be fixed in the next non-compatible release.")]
+        public static readonly int[] EvtsError =
+        {
+            EvtJobTimedout,
+            EvtJobFailed,
+            EvtJobFailedOver,
+            EvtJobRejected,
+            EvtJobCancelled,
+            EvtTaskTimedout,
+            EvtTaskFailed,
+            EvtCacheRebalanceStarted,
+            EvtCacheRebalanceStopped
+        };
+
+        /// <summary>
+        /// All discovery events except for <see cref="EvtNodeMetricsUpdated" />. Subscription to <see 
+        /// cref="EvtNodeMetricsUpdated" /> can generate massive amount of event processing in most cases is not 
+        /// necessary. If this event is indeed required you can subscribe to it individually or use <see 
+        /// cref="EvtsDiscoveryAll" /> array.
+        /// </summary>
+        [SuppressMessage("Microsoft.Security", "CA2105:ArrayFieldsShouldNotBeReadOnly",
+            Justification = "Breaking change. Should be fixed in the next non-compatible release.")]
+        public static readonly int[] EvtsDiscovery =
+        {
+            EvtNodeJoined,
+            EvtNodeLeft,
+            EvtNodeFailed,
+            EvtNodeSegmented,
+            EvtClientNodeDisconnected,
+            EvtClientNodeReconnected
+        };
+
+        /// <summary>
+        /// All discovery events.
+        /// </summary>
+        [SuppressMessage("Microsoft.Security", "CA2105:ArrayFieldsShouldNotBeReadOnly",
+            Justification = "Breaking change. Should be fixed in the next non-compatible release.")]
+        public static readonly int[] EvtsDiscoveryAll =
+        {
+            EvtNodeJoined,
+            EvtNodeLeft,
+            EvtNodeFailed,
+            EvtNodeSegmented,
+            EvtNodeMetricsUpdated,
+            EvtClientNodeDisconnected,
+            EvtClientNodeReconnected
+        };
+
+        /// <summary>
+        /// All Ignite job execution events.
+        /// </summary>
+        [SuppressMessage("Microsoft.Security", "CA2105:ArrayFieldsShouldNotBeReadOnly",
+            Justification = "Breaking change. Should be fixed in the next non-compatible release.")]
+        public static readonly int[] EvtsJobExecution =
+        {
+            EvtJobMapped,
+            EvtJobResulted,
+            EvtJobFailedOver,
+            EvtJobStarted,
+            EvtJobFinished,
+            EvtJobTimedout,
+            EvtJobRejected,
+            EvtJobFailed,
+            EvtJobQueued,
+            EvtJobCancelled
+        };
+
+        /// <summary>
+        /// All Ignite task execution events.
+        /// </summary>
+        [SuppressMessage("Microsoft.Security", "CA2105:ArrayFieldsShouldNotBeReadOnly",
+            Justification = "Breaking change. Should be fixed in the next non-compatible release.")]
+        public static readonly int[] EvtsTaskExecution =
+        {
+            EvtTaskStarted,
+            EvtTaskFinished,
+            EvtTaskFailed,
+            EvtTaskTimedout,
+            EvtTaskSessionAttrSet,
+            EvtTaskReduced
+        };
+
+        /// <summary>
+        /// All cache events.
+        /// </summary>
+        [SuppressMessage("Microsoft.Security", "CA2105:ArrayFieldsShouldNotBeReadOnly",
+            Justification = "Breaking change. Should be fixed in the next non-compatible release.")]
+        public static readonly int[] EvtsCache =
+        {
+            EvtCacheEntryCreated,
+            EvtCacheEntryDestroyed,
+            EvtCacheObjectPut,
+            EvtCacheObjectRead,
+            EvtCacheObjectRemoved,
+            EvtCacheObjectLocked,
+            EvtCacheObjectUnlocked,
+            EvtCacheObjectSwapped,
+            EvtCacheObjectUnswapped,
+            EvtCacheObjectExpired
+        };
+
+        /// <summary>
+        /// All cache rebalance events.
+        /// </summary>
+        [SuppressMessage("Microsoft.Security", "CA2105:ArrayFieldsShouldNotBeReadOnly",
+            Justification = "Breaking change. Should be fixed in the next non-compatible release.")]
+        public static readonly int[] EvtsCacheRebalance =
+        {
+            EvtCacheRebalanceStarted,
+            EvtCacheRebalanceStopped,
+            EvtCacheRebalancePartLoaded,
+            EvtCacheRebalancePartUnloaded,
+            EvtCacheRebalanceObjectLoaded,
+            EvtCacheRebalanceObjectUnloaded,
+            EvtCacheRebalancePartDataLost
+        };
+
+        /// <summary>
+        /// All cache lifecycle events.
+        /// </summary>
+        [SuppressMessage("Microsoft.Security", "CA2105:ArrayFieldsShouldNotBeReadOnly",
+            Justification = "Breaking change. Should be fixed in the next non-compatible release.")]
+        public static readonly int[] EvtsCacheLifecycle =
+        {
+            EvtCacheStarted,
+            EvtCacheStopped,
+            EvtCacheNodesLeft
+        };
+
+        /// <summary>
+        /// All cache query events.
+        /// </summary>
+        [SuppressMessage("Microsoft.Security", "CA2105:ArrayFieldsShouldNotBeReadOnly",
+            Justification = "Breaking change. Should be fixed in the next non-compatible release.")]
+        public static readonly int[] EvtsCacheQuery =
+        {
+            EvtCacheQueryExecuted,
+            EvtCacheQueryObjectRead
+        };
+
+        /// <summary>
+        /// All swap space events.
+        /// </summary>
+        [SuppressMessage("Microsoft.Security", "CA2105:ArrayFieldsShouldNotBeReadOnly",
+            Justification = "Breaking change. Should be fixed in the next non-compatible release.")]
+        public static readonly int[] EvtsSwapspace =
+        {
+            EvtSwapSpaceCleared,
+            EvtSwapSpaceDataRemoved,
+            EvtSwapSpaceDataRead,
+            EvtSwapSpaceDataStored,
+            EvtSwapSpaceDataEvicted
+        };
+
+        /// <summary>
+        /// All Ignite events.
+        /// </summary>
+        [SuppressMessage("Microsoft.Security", "CA2105:ArrayFieldsShouldNotBeReadOnly",
+            Justification = "Breaking change. Should be fixed in the next non-compatible release.")]
+        public static readonly int[] EvtsAll = GetAllEvents();
+
+        /// <summary>
+        /// All Ignite events (<b>excluding</b> metric update event).
+        /// </summary>
+        [SuppressMessage("Microsoft.Security", "CA2105:ArrayFieldsShouldNotBeReadOnly",
+            Justification = "Breaking change. Should be fixed in the next non-compatible release.")]
+        public static readonly int[] EvtsAllMinusMetricUpdate =
+            EvtsAll.Where(x => x != EvtNodeMetricsUpdated).ToArray();
+
+        /// <summary>
+        /// Gets all the events.
+        /// </summary>
+        /// <returns>All event ids.</returns>
+        private static int[] GetAllEvents()
+        {
+            return typeof (EventType).GetFields(BindingFlags.Public | BindingFlags.Static)
+                .Where(x => x.FieldType == typeof (int))
+                .Select(x => (int) x.GetValue(null)).ToArray();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/IEvent.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/IEvent.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/IEvent.cs
new file mode 100644
index 0000000..181aeef
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/IEvent.cs
@@ -0,0 +1,74 @@
+/*
+ * 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.Events
+{
+    using System;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// Represents a Ignite event.
+    /// </summary>
+    public interface IEvent
+    {
+        /// <summary>
+        /// Gets globally unique ID of this event.
+        /// </summary>
+        IgniteGuid Id { get; }
+
+        /// <summary>
+        /// Gets locally unique ID that is atomically incremented for each event. Unlike global <see cref="Id" />
+        /// this local ID can be used for ordering events on this node. 
+        /// <para/> 
+        /// Note that for performance considerations Ignite doesn't order events globally.
+        /// </summary>
+        long LocalOrder { get; }
+
+        /// <summary>
+        /// Node where event occurred and was recorded.
+        /// </summary>
+        IClusterNode Node { get; }
+
+        /// <summary>
+        /// Gets optional message for this event.
+        /// </summary>
+        string Message { get; }
+
+        /// <summary>
+        /// Gets type of this event. All system event types are defined in <see cref="EventType"/>
+        /// </summary>
+        int Type { get; }
+
+        /// <summary>
+        /// Gets name of this event.
+        /// </summary>
+        string Name { get; }
+
+        /// <summary>
+        /// Gets event timestamp. Timestamp is local to the node on which this event was produced. 
+        /// Note that more than one event can be generated with the same timestamp. 
+        /// For ordering purposes use <see cref="LocalOrder"/> instead.
+        /// </summary>
+        DateTime TimeStamp { get; }
+
+        /// <summary>
+        /// Gets shortened version of ToString result.
+        /// </summary>
+        string ToShortString();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/IEventFilter.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/IEventFilter.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/IEventFilter.cs
new file mode 100644
index 0000000..7523c52
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/IEventFilter.cs
@@ -0,0 +1,36 @@
+/*
+ * 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.Events
+{
+    using System;
+
+    /// <summary>
+    /// Represents an event filter.
+    /// </summary>
+    /// <typeparam name="T">Event type.</typeparam>
+    public interface IEventFilter<in T> where T : IEvent
+    {
+        /// <summary>
+        /// Determines whether specified event passes this filtger.
+        /// </summary>
+        /// <param name="nodeId">Node identifier.</param>
+        /// <param name="evt">Event.</param>
+        /// <returns>Value indicating whether specified event passes this filtger.</returns>
+        bool Invoke(Guid nodeId, T evt);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/IEvents.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/IEvents.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/IEvents.cs
new file mode 100644
index 0000000..e13513c
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/IEvents.cs
@@ -0,0 +1,182 @@
+/*
+ * 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.Events
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Diagnostics.CodeAnalysis;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// Provides functionality for local and remote event notifications on nodes defined by <see cref="ClusterGroup"/>.
+    /// <para/>
+    /// All members are thread-safe and may be used concurrently from multiple threads.
+    /// </summary>
+    public interface IEvents : IAsyncSupport<IEvents>
+    {
+        /// <summary>
+        /// Gets the cluster group to which this instance belongs.
+        /// </summary>
+        IClusterGroup ClusterGroup { get; }
+
+        /// <summary>
+        /// Queries nodes in this cluster group for events using passed in predicate filter for event selection.
+        /// </summary>
+        /// <typeparam name="T">Type of events.</typeparam>
+        /// <param name="filter">Predicate filter used to query events on remote nodes.</param>
+        /// <param name="timeout">Maximum time to wait for result, null or 0 to wait forever.</param>
+        /// <param name="types">Event types to be queried.</param>
+        /// <returns>Collection of Ignite events returned from specified nodes.</returns>
+        [AsyncSupported]
+        [SuppressMessage("Microsoft.Design", "CA1002:DoNotExposeGenericLists")]
+        List<T> RemoteQuery<T>(IEventFilter<T> filter, TimeSpan? timeout = null, params int[] types) 
+            where T : IEvent;
+
+        /// <summary>
+        /// Adds event listener for specified events to all nodes in the cluster group (possibly including local node 
+        /// if it belongs to the cluster group as well). This means that all events occurring on any node within this 
+        /// cluster group that pass remote filter will be sent to local node for local listener notifications.
+        /// <para/>
+        /// The listener can be unsubscribed automatically if local node stops, if localListener callback 
+        /// returns false or if <see cref="StopRemoteListen"/> is called.
+        /// </summary>
+        /// <typeparam name="T">Type of events.</typeparam>
+        /// <param name="bufSize">Remote events buffer size. Events from remote nodes won't be sent until buffer
+        /// is full or time interval is exceeded.</param>
+        /// <param name="interval">Maximum time interval after which events from remote node will be sent. Events
+        /// from remote nodes won't be sent until buffer is full or time interval is exceeded.</param>
+        /// <param name="autoUnsubscribe">Flag indicating that event listeners on remote nodes should be automatically 
+        /// unregistered if master node (node that initiated event listening) leaves topology. 
+        /// If this flag is false, listeners will be unregistered only when <see cref="StopRemoteListen"/>
+        /// method is called, or the localListener returns false.</param>
+        /// <param name="localListener"> Listener callback that is called on local node. If null, these events will 
+        /// be handled on remote nodes by passed in remoteFilter.</param>
+        /// <param name="remoteFilter">
+        /// Filter callback that is called on remote node. Only events that pass the remote filter will be 
+        /// sent to local node. If null, all events of specified types will be sent to local node. 
+        /// This remote filter can be used to pre-handle events remotely, before they are passed in to local callback.
+        /// It will be auto-unsubscribed on the node where event occurred in case if it returns false.
+        /// </param>
+        /// <param name="types">
+        /// Types of events to listen for. If not provided, all events that pass the provided remote filter 
+        /// will be sent to local node.
+        /// </param>
+        /// <returns>
+        /// Operation ID that can be passed to <see cref="StopRemoteListen"/> method to stop listening.
+        /// </returns>
+        [AsyncSupported]
+        Guid RemoteListen<T>(int bufSize = 1, TimeSpan? interval = null, bool autoUnsubscribe = true,
+            IEventFilter<T> localListener = null, IEventFilter<T> remoteFilter = null, params int[] types) 
+            where T : IEvent;
+
+        /// <summary>
+        /// Stops listening to remote events. This will unregister all listeners identified with provided operation ID 
+        /// on all nodes defined by <see cref="ClusterGroup"/>.
+        /// </summary>
+        /// <param name="opId">Operation ID that was returned from <see cref="RemoteListen{T}"/>.</param>
+        [AsyncSupported]
+        void StopRemoteListen(Guid opId);
+
+        /// <summary>
+        /// Waits for the specified events.
+        /// </summary>
+        /// <param name="types">Types of the events to wait for. 
+        /// If not provided, all events will be passed to the filter.</param>
+        /// <returns>Ignite event.</returns>
+        [AsyncSupported]
+        IEvent WaitForLocal(params int[] types);
+
+        /// <summary>
+        /// Waits for the specified events.
+        /// </summary>
+        /// <typeparam name="T">Type of events.</typeparam>
+        /// <param name="filter">Optional filtering predicate. Event wait will end as soon as it returns false.</param>
+        /// <param name="types">Types of the events to wait for. 
+        /// If not provided, all events will be passed to the filter.</param>
+        /// <returns>Ignite event.</returns>
+        [AsyncSupported]
+        T WaitForLocal<T>(IEventFilter<T> filter, params int[] types) where T : IEvent;
+
+        /// <summary>
+        /// Queries local node for events using of specified types.
+        /// </summary>
+        /// <param name="types">Event types to be queried. Optional.</param>
+        /// <returns>Collection of Ignite events found on local node.</returns>
+        [SuppressMessage("Microsoft.Design", "CA1002:DoNotExposeGenericLists")]
+        List<IEvent> LocalQuery(params int[] types);
+
+        /// <summary>
+        /// Records customer user generated event. All registered local listeners will be notified.
+        /// <para/>
+        /// NOTE: all types in range <b>from 1 to 1000 are reserved</b> for
+        /// internal Ignite events and should not be used by user-defined events.
+        /// Attempt to record internal event with this method will cause <see cref="ArgumentException"/> to be thrown.
+        /// </summary>
+        /// <param name="evt">Locally generated event.</param>
+        /// <exception cref="ArgumentException">If event type is within Ignite reserved range (1 � 1000)</exception>
+        void RecordLocal(IEvent evt);
+
+        /// <summary>
+        /// Adds an event listener for local events. Note that listener will be added regardless of whether 
+        /// local node is in this cluster group or not.
+        /// </summary>
+        /// <typeparam name="T">Type of events.</typeparam>
+        /// <param name="listener">Predicate that is called on each received event. If predicate returns false,
+        /// it will be unregistered and will stop receiving events.</param>
+        /// <param name="types">Event types for which this listener will be notified, should not be empty.</param>
+        void LocalListen<T>(IEventFilter<T> listener, params int[] types) where T : IEvent;
+
+        /// <summary>
+        /// Removes local event listener.
+        /// </summary>
+        /// <typeparam name="T">Type of events.</typeparam>
+        /// <param name="listener">Local event listener to remove.</param>
+        /// <param name="types">Types of events for which to remove listener. If not specified, then listener
+        /// will be removed for all types it was registered for.</param>
+        /// <returns>True if listener was removed, false otherwise.</returns>
+        bool StopLocalListen<T>(IEventFilter<T> listener, params int[] types) where T : IEvent;
+
+        /// <summary>
+        /// Enables provided events. Allows to start recording events that were disabled before. 
+        /// Note that provided events will be enabled regardless of whether local node is in this cluster group or not.
+        /// </summary>
+        /// <param name="types">Events to enable.</param>
+        void EnableLocal(params int[] types);
+
+        /// <summary>
+        /// Disables provided events. Allows to stop recording events that were enabled before. Note that specified 
+        /// events will be disabled regardless of whether local node is in this cluster group or not.
+        /// </summary>
+        /// <param name="types">Events to disable.</param>
+        void DisableLocal(params int[] types);
+
+        /// <summary>
+        /// Gets types of enabled events.
+        /// </summary>
+        /// <returns>Types of enabled events.</returns>
+        int[] GetEnabledEvents();
+
+        /// <summary>
+        /// Determines whether the specified event is enabled.
+        /// </summary>
+        /// <param name="type">Event type.</param>
+        /// <returns>Value indicating whether the specified event is enabled.</returns>
+        bool IsEnabled(int type);
+    }
+}
\ No newline at end of file


[23/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableMode.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableMode.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableMode.cs
new file mode 100644
index 0000000..670b091
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableMode.cs
@@ -0,0 +1,40 @@
+/*
+ * 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.Portable
+{
+    /// <summary>
+    /// Portable mode.
+    /// </summary>
+    internal enum PortableMode
+    {
+        /// <summary>
+        /// Deserialize top-level portable objects, but leave nested portable objects in portable form.
+        /// </summary>
+        Deserialize,
+
+        /// <summary>
+        /// Keep portable objects in portable form.
+        /// </summary>
+        KeepPortable,
+
+        /// <summary>
+        /// Always return IPortableObject.
+        /// </summary>
+        ForcePortable
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectHandle.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectHandle.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectHandle.cs
new file mode 100644
index 0000000..f2c3842
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectHandle.cs
@@ -0,0 +1,59 @@
+/*
+ * 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.Portable
+{
+    /// <summary>
+    /// Object handle. Wraps a single value.
+    /// </summary>
+    internal class PortableObjectHandle
+    {
+        /** Value. */
+        private readonly object _val;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="PortableObjectHandle"/> class.
+        /// </summary>
+        /// <param name="val">The value.</param>
+        public PortableObjectHandle(object val)
+        {
+            _val = val;
+        }
+
+        /// <summary>
+        /// Gets the value.
+        /// </summary>
+        public object Value
+        {
+            get { return _val; }
+        }
+
+        /** <inheritdoc /> */
+        public override bool Equals(object obj)
+        {
+            var that = obj as PortableObjectHandle;
+
+            return that != null && _val == that._val;
+        }
+
+        /** <inheritdoc /> */
+        public override int GetHashCode()
+        {
+            return _val != null ? _val.GetHashCode() : 0;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableOrSerializableObjectHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableOrSerializableObjectHolder.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableOrSerializableObjectHolder.cs
new file mode 100644
index 0000000..06ccf8b
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableOrSerializableObjectHolder.cs
@@ -0,0 +1,66 @@
+/*
+ * 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.Portable
+{
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Wraps portable/serializable item in a portable.
+    /// </summary>
+    internal class PortableOrSerializableObjectHolder : IPortableWriteAware
+    {
+        /** */
+        private readonly object _item;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="SerializableObjectHolder"/> class.
+        /// </summary>
+        /// <param name="item">The item to wrap.</param>
+        public PortableOrSerializableObjectHolder(object item)
+        {
+            _item = item;
+        }
+
+        /// <summary>
+        /// Gets or sets the item to wrap.
+        /// </summary>
+        public object Item
+        {
+            get { return _item; }
+        }
+
+        /** <inheritDoc /> */
+        public void WritePortable(IPortableWriter writer)
+        {
+            var writer0 = (PortableWriterImpl)writer.RawWriter();
+
+            writer0.DetachNext();
+
+            PortableUtils.WritePortableOrSerializable(writer0, Item);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="PortableOrSerializableObjectHolder"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        public PortableOrSerializableObjectHolder(IPortableReader reader)
+        {
+            _item = PortableUtils.ReadPortableOrSerializable<object>((PortableReaderImpl)reader.RawReader());
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableReaderHandleDictionary.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableReaderHandleDictionary.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableReaderHandleDictionary.cs
new file mode 100644
index 0000000..6a765c3
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableReaderHandleDictionary.cs
@@ -0,0 +1,42 @@
+/*
+ * 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.Portable
+{
+    /// <summary>
+    /// Object handle dictionary for PortableReader.
+    /// </summary>
+    internal class PortableReaderHandleDictionary : PortableHandleDictionary<int, object>
+    {
+        /// <summary>
+        /// Constructor with initial key-value pair.
+        /// </summary>
+        /// <param name="key">Key.</param>
+        /// <param name="val">Value.</param>
+        public PortableReaderHandleDictionary(int key, object val)
+            : base(key, val)
+        {
+            // No-op.
+        }
+
+        /** <inheritdoc /> */
+        protected override int EmptyKey
+        {
+            get { return -1; }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableReaderImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableReaderImpl.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableReaderImpl.cs
new file mode 100644
index 0000000..176ca27
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableReaderImpl.cs
@@ -0,0 +1,1013 @@
+/*
+ * 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.Portable
+{
+    using System;
+    using System.Collections;
+    using System.Collections.Generic;
+    using System.Diagnostics.CodeAnalysis;
+    using System.IO;
+    using System.Runtime.Serialization;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Portable reader implementation. 
+    /// </summary>
+    internal class PortableReaderImpl : IPortableReader, IPortableRawReader
+    {
+        /** Marshaller. */
+        private readonly PortableMarshaller _marsh;
+
+        /** Type descriptors. */
+        private readonly IDictionary<long, IPortableTypeDescriptor> _descs;
+
+        /** Parent builder. */
+        private readonly PortableBuilderImpl _builder;
+
+        /** Handles. */
+        private PortableReaderHandleDictionary _hnds;
+
+        /** Current type ID. */
+        private int _curTypeId;
+
+        /** Current position. */
+        private int _curPos;
+
+        /** Current raw data offset. */
+        private int _curRawOffset;
+
+        /** Current converter. */
+        private IPortableNameMapper _curConverter;
+
+        /** Current mapper. */
+        private IPortableIdMapper _curMapper;
+
+        /** Current raw flag. */
+        private bool _curRaw;
+
+        /** Detach flag. */
+        private bool _detach;
+
+        /** Portable read mode. */
+        private PortableMode _mode;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="marsh">Marshaller.</param>
+        /// <param name="descs">Descriptors.</param>
+        /// <param name="stream">Input stream.</param>
+        /// <param name="mode">The mode.</param>
+        /// <param name="builder">Builder.</param>
+        public PortableReaderImpl
+            (PortableMarshaller marsh,
+            IDictionary<long, IPortableTypeDescriptor> descs, 
+            IPortableStream stream, 
+            PortableMode mode,
+            PortableBuilderImpl builder)
+        {
+            _marsh = marsh;
+            _descs = descs;
+            _mode = mode;
+            _builder = builder;
+
+            Stream = stream;
+        }
+
+        /// <summary>
+        /// Gets the marshaller.
+        /// </summary>
+        public PortableMarshaller Marshaller
+        {
+            get { return _marsh; }
+        }
+
+        /** <inheritdoc /> */
+        public IPortableRawReader RawReader()
+        {
+            MarkRaw();
+
+            return this;
+        }
+
+        /** <inheritdoc /> */
+        public bool ReadBoolean(string fieldName)
+        {
+            return ReadField(fieldName, r => r.ReadBoolean());
+        }
+
+        /** <inheritdoc /> */
+        public bool ReadBoolean()
+        {
+            return Stream.ReadBool();
+        }
+
+        /** <inheritdoc /> */
+        public bool[] ReadBooleanArray(string fieldName)
+        {
+            return ReadField(fieldName, PortableUtils.ReadBooleanArray);
+        }
+
+        /** <inheritdoc /> */
+        public bool[] ReadBooleanArray()
+        {
+            return Read(PortableUtils.ReadBooleanArray);
+        }
+
+        /** <inheritdoc /> */
+        public byte ReadByte(string fieldName)
+        {
+            return ReadField(fieldName, ReadByte);
+        }
+
+        /** <inheritdoc /> */
+        public byte ReadByte()
+        {
+            return Stream.ReadByte();
+        }
+
+        /** <inheritdoc /> */
+        public byte[] ReadByteArray(string fieldName)
+        {
+            return ReadField(fieldName, PortableUtils.ReadByteArray);
+        }
+
+        /** <inheritdoc /> */
+        public byte[] ReadByteArray()
+        {
+            return Read(PortableUtils.ReadByteArray);
+        }
+
+        /** <inheritdoc /> */
+        public short ReadShort(string fieldName)
+        {
+            return ReadField(fieldName, ReadShort);
+        }
+
+        /** <inheritdoc /> */
+        public short ReadShort()
+        {
+            return Stream.ReadShort();
+        }
+
+        /** <inheritdoc /> */
+        public short[] ReadShortArray(string fieldName)
+        {
+            return ReadField(fieldName, PortableUtils.ReadShortArray);
+        }
+
+        /** <inheritdoc /> */
+        public short[] ReadShortArray()
+        {
+            return Read(PortableUtils.ReadShortArray);
+        }
+
+        /** <inheritdoc /> */
+        public char ReadChar(string fieldName)
+        {
+            return ReadField(fieldName, ReadChar);
+        }
+
+        /** <inheritdoc /> */
+        public char ReadChar()
+        {
+            return Stream.ReadChar();
+        }
+
+        /** <inheritdoc /> */
+        public char[] ReadCharArray(string fieldName)
+        {
+            return ReadField(fieldName, PortableUtils.ReadCharArray);
+        }
+
+        /** <inheritdoc /> */
+        public char[] ReadCharArray()
+        {
+            return Read(PortableUtils.ReadCharArray);
+        }
+
+        /** <inheritdoc /> */
+        public int ReadInt(string fieldName)
+        {
+            return ReadField(fieldName, ReadInt);
+        }
+
+        /** <inheritdoc /> */
+        public int ReadInt()
+        {
+            return Stream.ReadInt();
+        }
+
+        /** <inheritdoc /> */
+        public int[] ReadIntArray(string fieldName)
+        {
+            return ReadField(fieldName, PortableUtils.ReadIntArray);
+        }
+
+        /** <inheritdoc /> */
+        public int[] ReadIntArray()
+        {
+            return Read(PortableUtils.ReadIntArray);
+        }
+
+        /** <inheritdoc /> */
+        public long ReadLong(string fieldName)
+        {
+            return ReadField(fieldName, ReadLong);
+        }
+
+        /** <inheritdoc /> */
+        public long ReadLong()
+        {
+            return Stream.ReadLong();
+        }
+
+        /** <inheritdoc /> */
+        public long[] ReadLongArray(string fieldName)
+        {
+            return ReadField(fieldName, PortableUtils.ReadLongArray);
+        }
+
+        /** <inheritdoc /> */
+        public long[] ReadLongArray()
+        {
+            return Read(PortableUtils.ReadLongArray);
+        }
+
+        /** <inheritdoc /> */
+        public float ReadFloat(string fieldName)
+        {
+            return ReadField(fieldName, ReadFloat);
+        }
+
+        /** <inheritdoc /> */
+        public float ReadFloat()
+        {
+            return Stream.ReadFloat();
+        }
+
+        /** <inheritdoc /> */
+        public float[] ReadFloatArray(string fieldName)
+        {
+            return ReadField(fieldName, PortableUtils.ReadFloatArray);
+        }
+
+        /** <inheritdoc /> */
+        public float[] ReadFloatArray()
+        {
+            return Read(PortableUtils.ReadFloatArray);
+        }
+
+        /** <inheritdoc /> */
+        public double ReadDouble(string fieldName)
+        {
+            return ReadField(fieldName, ReadDouble);
+        }
+
+        /** <inheritdoc /> */
+        public double ReadDouble()
+        {
+            return Stream.ReadDouble();
+        }
+
+        /** <inheritdoc /> */
+        public double[] ReadDoubleArray(string fieldName)
+        {
+            return ReadField(fieldName, PortableUtils.ReadDoubleArray);
+        }
+
+        /** <inheritdoc /> */
+        public double[] ReadDoubleArray()
+        {
+            return Read(PortableUtils.ReadDoubleArray);
+        }
+
+        /** <inheritdoc /> */
+        public decimal ReadDecimal(string fieldName)
+        {
+            return ReadField(fieldName, PortableUtils.ReadDecimal);
+        }
+
+        /** <inheritdoc /> */
+        public decimal ReadDecimal()
+        {
+            return Read(PortableUtils.ReadDecimal);
+        }
+
+        /** <inheritdoc /> */
+        public decimal[] ReadDecimalArray(string fieldName)
+        {
+            return ReadField(fieldName, PortableUtils.ReadDecimalArray);
+        }
+
+        /** <inheritdoc /> */
+        public decimal[] ReadDecimalArray()
+        {
+            return Read(PortableUtils.ReadDecimalArray);
+        }
+
+        /** <inheritdoc /> */
+        public DateTime? ReadDate(string fieldName)
+        {
+            return ReadDate(fieldName, false);
+        }
+
+        /** <inheritdoc /> */
+        public DateTime? ReadDate(string fieldName, bool local)
+        {
+            return ReadField(fieldName, r => PortableUtils.ReadDate(r, local));
+        }
+
+        /** <inheritdoc /> */
+        public DateTime? ReadDate()
+        {
+            return ReadDate(false);
+        }
+
+        /** <inheritdoc /> */
+        public DateTime? ReadDate(bool local)
+        {
+            return Read(r => PortableUtils.ReadDate(r, local));
+        }
+
+        /** <inheritdoc /> */
+        public DateTime?[] ReadDateArray(string fieldName)
+        {
+            return ReadDateArray(fieldName, false);
+        }
+
+        /** <inheritdoc /> */
+        public DateTime?[] ReadDateArray(string fieldName, bool local)
+        {
+            return ReadField(fieldName, r => PortableUtils.ReadDateArray(r, local));
+        }
+
+        /** <inheritdoc /> */
+        public DateTime?[] ReadDateArray()
+        {
+            return ReadDateArray(false);
+        }
+
+        /** <inheritdoc /> */
+        public DateTime?[] ReadDateArray(bool local)
+        {
+            return Read(r => PortableUtils.ReadDateArray(r, local));
+        }
+
+        /** <inheritdoc /> */
+        public string ReadString(string fieldName)
+        {
+            return ReadField(fieldName, PortableUtils.ReadString);
+        }
+
+        /** <inheritdoc /> */
+        public string ReadString()
+        {
+            return Read(PortableUtils.ReadString);
+        }
+
+        /** <inheritdoc /> */
+        public string[] ReadStringArray(string fieldName)
+        {
+            return ReadField(fieldName, r => PortableUtils.ReadGenericArray<string>(r, false));
+        }
+
+        /** <inheritdoc /> */
+        public string[] ReadStringArray()
+        {
+            return Read(r => PortableUtils.ReadGenericArray<string>(r, false));
+        }
+
+        /** <inheritdoc /> */
+        public Guid? ReadGuid(string fieldName)
+        {
+            return ReadField(fieldName, PortableUtils.ReadGuid);
+        }
+
+        /** <inheritdoc /> */
+        public Guid? ReadGuid()
+        {
+            return Read(PortableUtils.ReadGuid);
+        }
+
+        /** <inheritdoc /> */
+        public Guid?[] ReadGuidArray(string fieldName)
+        {
+            return ReadField(fieldName, r => PortableUtils.ReadGenericArray<Guid?>(r, false));
+        }
+
+        /** <inheritdoc /> */
+        public Guid?[] ReadGuidArray()
+        {
+            return Read(r => PortableUtils.ReadGenericArray<Guid?>(r, false));
+        }
+
+        /** <inheritdoc /> */
+        public T ReadEnum<T>(string fieldName)
+        {
+            return ReadField(fieldName, PortableUtils.ReadEnum<T>);
+        }
+
+        /** <inheritdoc /> */
+        public T ReadEnum<T>()
+        {
+            return Read(PortableUtils.ReadEnum<T>);
+        }
+
+        /** <inheritdoc /> */
+        public T[] ReadEnumArray<T>(string fieldName)
+        {
+            return ReadField(fieldName, r => PortableUtils.ReadGenericArray<T>(r, true));
+        }
+
+        /** <inheritdoc /> */
+        public T[] ReadEnumArray<T>()
+        {
+            return Read(r => PortableUtils.ReadGenericArray<T>(r, true));
+        }
+
+        /** <inheritdoc /> */
+        public T ReadObject<T>(string fieldName)
+        {
+            if (_curRaw)
+                throw new PortableException("Cannot read named fields after raw data is read.");
+
+            int fieldId = PortableUtils.FieldId(_curTypeId, fieldName, _curConverter, _curMapper);
+
+            if (SeekField(fieldId))
+                return Deserialize<T>();
+
+            return default(T);
+        }
+
+        /** <inheritdoc /> */
+        public T ReadObject<T>()
+        {
+            return Deserialize<T>();
+        }
+
+        /** <inheritdoc /> */
+        public T[] ReadObjectArray<T>(string fieldName)
+        {
+            return ReadField(fieldName, r => PortableUtils.ReadGenericArray<T>(r, true));
+        }
+
+        /** <inheritdoc /> */
+        public T[] ReadObjectArray<T>()
+        {
+            return Read(r => PortableUtils.ReadGenericArray<T>(r, true));
+        }
+
+        /** <inheritdoc /> */
+        public ICollection ReadCollection(string fieldName)
+        {
+            return ReadCollection(fieldName, null, null);
+        }
+
+        /** <inheritdoc /> */
+        public ICollection ReadCollection()
+        {
+            return ReadCollection(null, null);
+        }
+
+        /** <inheritdoc /> */
+        public ICollection ReadCollection(string fieldName, PortableCollectionFactory factory,
+            PortableCollectionAdder adder)
+        {
+            return ReadField(fieldName, r => PortableUtils.ReadCollection(r, factory, adder));
+        }
+
+        /** <inheritdoc /> */
+        public ICollection ReadCollection(PortableCollectionFactory factory,
+            PortableCollectionAdder adder)
+        {
+            return Read(r => PortableUtils.ReadCollection(r, factory, adder));
+        }
+
+        /** <inheritdoc /> */
+        public ICollection<T> ReadGenericCollection<T>(string fieldName)
+        {
+            return ReadGenericCollection<T>(fieldName, null);
+        }
+
+        /** <inheritdoc /> */
+        public ICollection<T> ReadGenericCollection<T>()
+        {
+            return ReadGenericCollection((PortableGenericCollectionFactory<T>) null);
+        }
+
+        /** <inheritdoc /> */
+        public ICollection<T> ReadGenericCollection<T>(string fieldName,
+            PortableGenericCollectionFactory<T> factory)
+        {
+            return ReadField(fieldName, r => PortableUtils.ReadGenericCollection(r, factory));
+        }
+
+        /** <inheritdoc /> */
+        public ICollection<T> ReadGenericCollection<T>(PortableGenericCollectionFactory<T> factory)
+        {
+            return Read(r => PortableUtils.ReadGenericCollection(r, factory));
+        }
+
+        /** <inheritdoc /> */
+        public IDictionary ReadDictionary(string fieldName)
+        {
+            return ReadDictionary(fieldName, null);
+        }
+
+        /** <inheritdoc /> */
+        public IDictionary ReadDictionary()
+        {
+            return ReadDictionary((PortableDictionaryFactory)null);
+        }
+
+        /** <inheritdoc /> */
+        public IDictionary ReadDictionary(string fieldName, PortableDictionaryFactory factory)
+        {
+            return ReadField(fieldName, r => PortableUtils.ReadDictionary(r, factory));
+        }
+
+        /** <inheritdoc /> */
+        public IDictionary ReadDictionary(PortableDictionaryFactory factory)
+        {
+            return Read(r => PortableUtils.ReadDictionary(r, factory));
+        }
+
+        /** <inheritdoc /> */
+        public IDictionary<TK, TV> ReadGenericDictionary<TK, TV>(string fieldName)
+        {
+            return ReadGenericDictionary<TK, TV>(fieldName, null);
+        }
+
+        /** <inheritdoc /> */
+        public IDictionary<TK, TV> ReadGenericDictionary<TK, TV>()
+        {
+            return ReadGenericDictionary((PortableGenericDictionaryFactory<TK, TV>) null);
+        }
+
+        /** <inheritdoc /> */
+        public IDictionary<TK, TV> ReadGenericDictionary<TK, TV>(string fieldName,
+            PortableGenericDictionaryFactory<TK, TV> factory)
+        {
+            return ReadField(fieldName, r => PortableUtils.ReadGenericDictionary(r, factory));
+        }
+
+        /** <inheritdoc /> */
+        public IDictionary<TK, TV> ReadGenericDictionary<TK, TV>(PortableGenericDictionaryFactory<TK, TV> factory)
+        {
+            return Read(r => PortableUtils.ReadGenericDictionary(r, factory));
+        }
+
+        /// <summary>
+        /// Enable detach mode for the next object read. 
+        /// </summary>
+        public void DetachNext()
+        {
+            _detach = true;
+        }
+
+        /// <summary>
+        /// Deserialize object.
+        /// </summary>
+        /// <returns>Deserialized object.</returns>
+        public T Deserialize<T>()
+        {
+            int pos = Stream.Position;
+
+            byte hdr = Stream.ReadByte();
+
+            var doDetach = _detach;  // save detach flag into a var and reset so it does not go deeper
+
+            _detach = false;
+
+            switch (hdr)
+            {
+                case PortableUtils.HdrNull:
+                    return default(T);
+
+                case PortableUtils.HdrHnd:
+                    return ReadHandleObject<T>(pos);
+
+                case PortableUtils.HdrFull:
+                    return ReadFullObject<T>(pos);
+
+                case PortableUtils.TypePortable:
+                    return ReadPortableObject<T>(doDetach);
+            }
+
+            if (PortableUtils.IsPredefinedType(hdr))
+                return PortableSystemHandlers.ReadSystemType<T>(hdr, this);
+
+            throw new PortableException("Invalid header on deserialization [pos=" + pos + ", hdr=" + hdr + ']');
+        }
+
+        /// <summary>
+        /// Reads the portable object.
+        /// </summary>
+        private T ReadPortableObject<T>(bool doDetach)
+        {
+            var len = Stream.ReadInt();
+
+            var portablePos = Stream.Position;
+
+            if (_mode != PortableMode.Deserialize)
+                return TypeCaster<T>.Cast(ReadAsPortable(portablePos, len, doDetach));
+
+            Stream.Seek(len, SeekOrigin.Current);
+
+            var offset = Stream.ReadInt();
+
+            var retPos = Stream.Position;
+
+            Stream.Seek(portablePos + offset, SeekOrigin.Begin);
+
+            _mode = PortableMode.KeepPortable;
+
+            try
+            {
+                return Deserialize<T>();
+            }
+            finally
+            {
+                _mode = PortableMode.Deserialize;
+
+                Stream.Seek(retPos, SeekOrigin.Begin);
+            }
+        }
+
+        /// <summary>
+        /// Reads the portable object in portable form.
+        /// </summary>
+        private PortableUserObject ReadAsPortable(int dataPos, int dataLen, bool doDetach)
+        {
+            try
+            {
+                Stream.Seek(dataLen + dataPos, SeekOrigin.Begin);
+
+                var offs = Stream.ReadInt(); // offset inside data
+
+                var pos = dataPos + offs;
+
+                if (!doDetach)
+                    return GetPortableUserObject(pos, pos, Stream.Array());
+                
+                Stream.Seek(pos + 10, SeekOrigin.Begin);
+
+                var len = Stream.ReadInt();
+
+                Stream.Seek(pos, SeekOrigin.Begin);
+
+                return GetPortableUserObject(pos, 0, Stream.ReadByteArray(len));
+            }
+            finally
+            {
+                Stream.Seek(dataPos + dataLen + 4, SeekOrigin.Begin);
+            }
+        }
+
+        /// <summary>
+        /// Reads the full object.
+        /// </summary>
+        [SuppressMessage("Microsoft.Performance", "CA1804:RemoveUnusedLocals", MessageId = "hashCode")]
+        private T ReadFullObject<T>(int pos)
+        {
+            // Read header.
+            bool userType = Stream.ReadBool();
+            int typeId = Stream.ReadInt();
+            // ReSharper disable once UnusedVariable
+            int hashCode = Stream.ReadInt();
+            int len = Stream.ReadInt();
+            int rawOffset = Stream.ReadInt();
+
+            try
+            {
+                // Already read this object?
+                object hndObj;
+
+                if (_hnds != null && _hnds.TryGetValue(pos, out hndObj))
+                    return (T) hndObj;
+
+                if (userType && _mode == PortableMode.ForcePortable)
+                {
+                    PortableUserObject portObj;
+
+                    if (_detach)
+                    {
+                        Stream.Seek(pos, SeekOrigin.Begin);
+
+                        portObj = GetPortableUserObject(pos, 0, Stream.ReadByteArray(len));
+                    }
+                    else
+                        portObj = GetPortableUserObject(pos, pos, Stream.Array());
+
+                    T obj = _builder == null ? TypeCaster<T>.Cast(portObj) : TypeCaster<T>.Cast(_builder.Child(portObj));
+
+                    AddHandle(pos, obj);
+
+                    return obj;
+                }
+                else
+                {
+                    // Find descriptor.
+                    IPortableTypeDescriptor desc;
+
+                    if (!_descs.TryGetValue(PortableUtils.TypeKey(userType, typeId), out desc))
+                        throw new PortableException("Unknown type ID: " + typeId);
+
+                    // Instantiate object. 
+                    if (desc.Type == null)
+                        throw new PortableException("No matching type found for object [typeId=" +
+                                                    desc.TypeId + ", typeName=" + desc.TypeName + ']');
+
+                    // Preserve old frame.
+                    int oldTypeId = _curTypeId;
+                    int oldPos = _curPos;
+                    int oldRawOffset = _curRawOffset;
+                    IPortableNameMapper oldConverter = _curConverter;
+                    IPortableIdMapper oldMapper = _curMapper;
+                    bool oldRaw = _curRaw;
+
+                    // Set new frame.
+                    _curTypeId = typeId;
+                    _curPos = pos;
+                    _curRawOffset = rawOffset;
+                    _curConverter = desc.NameConverter;
+                    _curMapper = desc.Mapper;
+                    _curRaw = false;
+
+                    // Read object.
+                    object obj;
+
+                    var sysSerializer = desc.Serializer as IPortableSystemTypeSerializer;
+
+                    if (sysSerializer != null)
+                        obj = sysSerializer.ReadInstance(this);
+                    else
+                    {
+                        try
+                        {
+                            obj = FormatterServices.GetUninitializedObject(desc.Type);
+
+                            // Save handle.
+                            AddHandle(pos, obj);
+                        }
+                        catch (Exception e)
+                        {
+                            throw new PortableException("Failed to create type instance: " +
+                                                        desc.Type.AssemblyQualifiedName, e);
+                        }
+
+                        desc.Serializer.ReadPortable(obj, this);
+                    }
+
+                    // Restore old frame.
+                    _curTypeId = oldTypeId;
+                    _curPos = oldPos;
+                    _curRawOffset = oldRawOffset;
+                    _curConverter = oldConverter;
+                    _curMapper = oldMapper;
+                    _curRaw = oldRaw;
+
+                    var wrappedSerializable = obj as SerializableObjectHolder;
+
+                    return wrappedSerializable != null ? (T) wrappedSerializable.Item : (T) obj;
+                }
+            }
+            finally
+            {
+                // Advance stream pointer.
+                Stream.Seek(pos + len, SeekOrigin.Begin);
+            }
+        }
+
+        /// <summary>
+        /// Reads the handle object.
+        /// </summary>
+        private T ReadHandleObject<T>(int pos)
+        {
+            // Get handle position.
+            int hndPos = pos - Stream.ReadInt();
+
+            int retPos = Stream.Position;
+
+            try
+            {
+                object hndObj;
+
+                if (_builder == null || !_builder.CachedField(hndPos, out hndObj))
+                {
+                    if (_hnds == null || !_hnds.TryGetValue(hndPos, out hndObj))
+                    {
+                        // No such handler, i.e. we trying to deserialize inner object before deserializing outer.
+                        Stream.Seek(hndPos, SeekOrigin.Begin);
+
+                        hndObj = Deserialize<T>();
+                    }
+
+                    // Notify builder that we deserialized object on other location.
+                    if (_builder != null)
+                        _builder.CacheField(hndPos, hndObj);
+                }
+
+                return (T) hndObj;
+            }
+            finally
+            {
+                // Position stream to correct place.
+                Stream.Seek(retPos, SeekOrigin.Begin);
+            }
+        }
+
+        /// <summary>
+        /// Adds a handle to the dictionary.
+        /// </summary>
+        /// <param name="pos">Position.</param>
+        /// <param name="obj">Object.</param>
+        private void AddHandle(int pos, object obj)
+        {
+            if (_hnds == null)
+                _hnds = new PortableReaderHandleDictionary(pos, obj);
+            else
+                _hnds.Add(pos, obj);
+        }
+
+        /// <summary>
+        /// Underlying stream.
+        /// </summary>
+        public IPortableStream Stream
+        {
+            get;
+            private set;
+        }
+
+        /// <summary>
+        /// Mark current output as raw. 
+        /// </summary>
+        private void MarkRaw()
+        {
+            if (!_curRaw)
+            {
+                _curRaw = true;
+
+                Stream.Seek(_curPos + _curRawOffset, SeekOrigin.Begin);
+            }
+        }
+
+        /// <summary>
+        /// Seek field with the given ID in the current object.
+        /// </summary>
+        /// <param name="fieldId">Field ID.</param>
+        /// <returns>True in case the field was found and position adjusted, false otherwise.</returns>
+        private bool SeekField(int fieldId)
+        {
+            // This method is expected to be called when stream pointer is set either before
+            // the field or on raw data offset.
+            int start = _curPos + 18;
+            int end = _curPos + _curRawOffset;
+
+            int initial = Stream.Position;
+
+            int cur = initial;
+
+            while (cur < end)
+            {
+                int id = Stream.ReadInt();
+
+                if (fieldId == id)
+                {
+                    // Field is found, return.
+                    Stream.Seek(4, SeekOrigin.Current);
+
+                    return true;
+                }
+                
+                Stream.Seek(Stream.ReadInt(), SeekOrigin.Current);
+
+                cur = Stream.Position;
+            }
+
+            Stream.Seek(start, SeekOrigin.Begin);
+
+            cur = start;
+
+            while (cur < initial)
+            {
+                int id = Stream.ReadInt();
+
+                if (fieldId == id)
+                {
+                    // Field is found, return.
+                    Stream.Seek(4, SeekOrigin.Current);
+
+                    return true;
+                }
+                
+                Stream.Seek(Stream.ReadInt(), SeekOrigin.Current);
+
+                cur = Stream.Position;
+            }
+
+            return false;
+        }
+
+        /// <summary>
+        /// Determines whether header at current position is HDR_NULL.
+        /// </summary>
+        private bool IsNullHeader()
+        {
+            var hdr = ReadByte();
+
+            return hdr != PortableUtils.HdrNull;
+        }
+
+        /// <summary>
+        /// Seeks the field by name, reads header and returns true if field is present and header is not null.
+        /// </summary>
+        private bool SeekField(string fieldName)
+        {
+            if (_curRaw)
+                throw new PortableException("Cannot read named fields after raw data is read.");
+
+            var fieldId = PortableUtils.FieldId(_curTypeId, fieldName, _curConverter, _curMapper);
+
+            if (!SeekField(fieldId))
+                return false;
+
+            return IsNullHeader();
+        }
+
+        /// <summary>
+        /// Seeks specified field and invokes provided func.
+        /// </summary>
+        private T ReadField<T>(string fieldName, Func<IPortableStream, T> readFunc)
+        {
+            return SeekField(fieldName) ? readFunc(Stream) : default(T);
+        }
+
+        /// <summary>
+        /// Seeks specified field and invokes provided func.
+        /// </summary>
+        private T ReadField<T>(string fieldName, Func<PortableReaderImpl, T> readFunc)
+        {
+            return SeekField(fieldName) ? readFunc(this) : default(T);
+        }
+
+        /// <summary>
+        /// Seeks specified field and invokes provided func.
+        /// </summary>
+        private T ReadField<T>(string fieldName, Func<T> readFunc)
+        {
+            return SeekField(fieldName) ? readFunc() : default(T);
+        }
+
+        /// <summary>
+        /// Reads header and invokes specified func if the header is not null.
+        /// </summary>
+        private T Read<T>(Func<PortableReaderImpl, T> readFunc)
+        {
+            return IsNullHeader() ? readFunc(this) : default(T);
+        }
+
+        /// <summary>
+        /// Reads header and invokes specified func if the header is not null.
+        /// </summary>
+        private T Read<T>(Func<IPortableStream, T> readFunc)
+        {
+            return IsNullHeader() ? readFunc(Stream) : default(T);
+        }
+
+        /// <summary>
+        /// Gets the portable user object from a byte array.
+        /// </summary>
+        /// <param name="pos">Position in the current stream.</param>
+        /// <param name="offs">Offset in the byte array.</param>
+        /// <param name="bytes">Bytes.</param>
+        private PortableUserObject GetPortableUserObject(int pos, int offs, byte[] bytes)
+        {
+            Stream.Seek(pos + 2, SeekOrigin.Begin);
+
+            var id = Stream.ReadInt();
+
+            var hash = Stream.ReadInt();
+
+            return new PortableUserObject(_marsh, bytes, offs, id, hash);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableReflectiveRoutines.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableReflectiveRoutines.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableReflectiveRoutines.cs
new file mode 100644
index 0000000..d939d29
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableReflectiveRoutines.cs
@@ -0,0 +1,483 @@
+/*
+ * 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.Portable
+{
+    using System;
+    using System.Collections;
+    using System.Diagnostics;
+    using System.Linq.Expressions;
+    using System.Reflection;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Write action delegate.
+    /// </summary>
+    /// <param name="obj">Target object.</param>
+    /// <param name="writer">Writer.</param>
+    internal delegate void PortableReflectiveWriteAction(object obj, IPortableWriter writer);
+
+    /// <summary>
+    /// Read action delegate.
+    /// </summary>
+    /// <param name="obj">Target object.</param>
+    /// <param name="reader">Reader.</param>
+    internal delegate void PortableReflectiveReadAction(object obj, IPortableReader reader);
+
+    /// <summary>
+    /// Routines for reflective reads and writes.
+    /// </summary>
+    internal static class PortableReflectiveActions
+    {
+        /** Method: read enum. */
+        private static readonly MethodInfo MthdReadEnum =
+            typeof(IPortableReader).GetMethod("ReadEnum", new[] { typeof(string) });
+
+        /** Method: read enum array. */
+        private static readonly MethodInfo MthdReadEnumArray =
+            typeof(IPortableReader).GetMethod("ReadEnumArray", new[] { typeof(string) });
+
+        /** Method: read array. */
+        private static readonly MethodInfo MthdReadObjArray =
+            typeof(IPortableReader).GetMethod("ReadObjectArray", new[] { typeof(string) });
+
+        /** Method: read generic collection. */
+        private static readonly MethodInfo MthdReadGenericCollection =
+            typeof(IPortableReader).GetMethod("ReadGenericCollection", new[] { typeof(string) });
+
+        /** Method: read generic dictionary. */
+        private static readonly MethodInfo MthdReadGenericDictionary =
+            typeof(IPortableReader).GetMethod("ReadGenericDictionary", new[] { typeof(string) });
+
+        /** Method: read object. */
+        private static readonly MethodInfo MthdReadObj=
+            typeof(IPortableReader).GetMethod("ReadObject", new[] { typeof(string) });
+
+        /** Method: write enum array. */
+        private static readonly MethodInfo MthdWriteEnumArray =
+            typeof(IPortableWriter).GetMethod("WriteEnumArray");
+
+        /** Method: write array. */
+        private static readonly MethodInfo MthdWriteObjArray =
+            typeof(IPortableWriter).GetMethod("WriteObjectArray");
+
+        /** Method: write generic collection. */
+        private static readonly MethodInfo MthdWriteGenericCollection =
+            typeof(IPortableWriter).GetMethod("WriteGenericCollection");
+
+        /** Method: write generic dictionary. */
+        private static readonly MethodInfo MthdWriteGenericDictionary =
+            typeof(IPortableWriter).GetMethod("WriteGenericDictionary");
+
+        /** Method: read object. */
+        private static readonly MethodInfo MthdWriteObj =
+            typeof(IPortableWriter).GetMethod("WriteObject");
+
+        /// <summary>
+        /// Lookup read/write actions for the given type.
+        /// </summary>
+        /// <param name="field">The field.</param>
+        /// <param name="writeAction">Write action.</param>
+        /// <param name="readAction">Read action.</param>
+        public static void TypeActions(FieldInfo field, out PortableReflectiveWriteAction writeAction, 
+            out PortableReflectiveReadAction readAction)
+        {
+            var type = field.FieldType;
+
+            if (type.IsPrimitive)
+                HandlePrimitive(field, out writeAction, out readAction);
+            else if (type.IsArray)
+                HandleArray(field, out writeAction, out readAction);
+            else
+                HandleOther(field, out writeAction, out readAction);
+        }
+
+        /// <summary>
+        /// Handle primitive type.
+        /// </summary>
+        /// <param name="field">The field.</param>
+        /// <param name="writeAction">Write action.</param>
+        /// <param name="readAction">Read action.</param>
+        /// <exception cref="IgniteException">Unsupported primitive type:  + type.Name</exception>
+        private static void HandlePrimitive(FieldInfo field, out PortableReflectiveWriteAction writeAction,
+            out PortableReflectiveReadAction readAction)
+        {
+            var type = field.FieldType;
+
+            if (type == typeof(bool))
+            {
+                writeAction = GetWriter<bool>(field, (f, w, o) => w.WriteBoolean(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadBoolean(f));
+            }
+            else if (type == typeof(sbyte))
+            {
+                writeAction = GetWriter<sbyte>(field, (f, w, o) => w.WriteByte(f, unchecked((byte) o)));
+                readAction = GetReader(field, (f, r) => unchecked ((sbyte)r.ReadByte(f)));
+            }
+            else if (type == typeof(byte))
+            {
+                writeAction = GetWriter<byte>(field, (f, w, o) => w.WriteByte(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadByte(f));
+            }
+            else if (type == typeof(short))
+            {
+                writeAction = GetWriter<short>(field, (f, w, o) => w.WriteShort(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadShort(f));
+            }
+            else if (type == typeof(ushort))
+            {
+                writeAction = GetWriter<ushort>(field, (f, w, o) => w.WriteShort(f, unchecked((short) o)));
+                readAction = GetReader(field, (f, r) => unchecked((ushort) r.ReadShort(f)));
+            }
+            else if (type == typeof(char))
+            {
+                writeAction = GetWriter<char>(field, (f, w, o) => w.WriteChar(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadChar(f));
+            }
+            else if (type == typeof(int))
+            {
+                writeAction = GetWriter<int>(field, (f, w, o) => w.WriteInt(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadInt(f));
+            }
+            else if (type == typeof(uint))
+            {
+                writeAction = GetWriter<uint>(field, (f, w, o) => w.WriteInt(f, unchecked((int) o)));
+                readAction = GetReader(field, (f, r) => unchecked((uint) r.ReadInt(f)));
+            }
+            else if (type == typeof(long))
+            {
+                writeAction = GetWriter<long>(field, (f, w, o) => w.WriteLong(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadLong(f));
+            }
+            else if (type == typeof(ulong))
+            {
+                writeAction = GetWriter<ulong>(field, (f, w, o) => w.WriteLong(f, unchecked((long) o)));
+                readAction = GetReader(field, (f, r) => unchecked((ulong) r.ReadLong(f)));
+            }
+            else if (type == typeof(float))
+            {
+                writeAction = GetWriter<float>(field, (f, w, o) => w.WriteFloat(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadFloat(f));
+            }
+            else if (type == typeof(double))
+            {
+                writeAction = GetWriter<double>(field, (f, w, o) => w.WriteDouble(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadDouble(f));
+            }
+            else
+                throw new IgniteException("Unsupported primitive type: " + type.Name);
+        }
+
+        /// <summary>
+        /// Handle array type.
+        /// </summary>
+        /// <param name="field">The field.</param>
+        /// <param name="writeAction">Write action.</param>
+        /// <param name="readAction">Read action.</param>
+        private static void HandleArray(FieldInfo field, out PortableReflectiveWriteAction writeAction,
+            out PortableReflectiveReadAction readAction)
+        {
+            Type elemType = field.FieldType.GetElementType();
+
+            if (elemType == typeof(bool))
+            {
+                writeAction = GetWriter<bool[]>(field, (f, w, o) => w.WriteBooleanArray(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadBooleanArray(f));
+            }
+            else if (elemType == typeof(byte))
+            {
+                writeAction = GetWriter<byte[]>(field, (f, w, o) => w.WriteByteArray(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadByteArray(f));
+            }
+            else if (elemType == typeof(sbyte))
+            {
+                writeAction = GetWriter<sbyte[]>(field, (f, w, o) => w.WriteByteArray(f, (byte[]) (Array) o));
+                readAction = GetReader(field, (f, r) => (sbyte[]) (Array) r.ReadByteArray(f));
+            }
+            else if (elemType == typeof(short))
+            {
+                writeAction = GetWriter<short[]>(field, (f, w, o) => w.WriteShortArray(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadShortArray(f));
+            }
+            else if (elemType == typeof(ushort))
+            {
+                writeAction = GetWriter<ushort[]>(field, (f, w, o) => w.WriteShortArray(f, (short[]) (Array) o));
+                readAction = GetReader(field, (f, r) => (ushort[]) (Array) r.ReadShortArray(f));
+            }
+            else if (elemType == typeof(char))
+            {
+                writeAction = GetWriter<char[]>(field, (f, w, o) => w.WriteCharArray(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadCharArray(f));
+            }
+            else if (elemType == typeof(int))
+            {
+                writeAction = GetWriter<int[]>(field, (f, w, o) => w.WriteIntArray(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadIntArray(f));
+            }
+            else if (elemType == typeof(uint))
+            {
+                writeAction = GetWriter<uint[]>(field, (f, w, o) => w.WriteIntArray(f, (int[]) (Array) o));
+                readAction = GetReader(field, (f, r) => (uint[]) (Array) r.ReadIntArray(f));
+            }
+            else if (elemType == typeof(long))
+            {
+                writeAction = GetWriter<long[]>(field, (f, w, o) => w.WriteLongArray(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadLongArray(f));
+            }
+            else if (elemType == typeof(ulong))
+            {
+                writeAction = GetWriter<ulong[]>(field, (f, w, o) => w.WriteLongArray(f, (long[]) (Array) o));
+                readAction = GetReader(field, (f, r) => (ulong[]) (Array) r.ReadLongArray(f));
+            }
+            else if (elemType == typeof(float))
+            {
+                writeAction = GetWriter<float[]>(field, (f, w, o) => w.WriteFloatArray(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadFloatArray(f));
+            }
+            else if (elemType == typeof(double))
+            {
+                writeAction = GetWriter<double[]>(field, (f, w, o) => w.WriteDoubleArray(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadDoubleArray(f));
+            }
+            else if (elemType == typeof(decimal))
+            {
+                writeAction = GetWriter<decimal[]>(field, (f, w, o) => w.WriteDecimalArray(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadDecimalArray(f));
+            }
+            else if (elemType == typeof(string))
+            {
+                writeAction = GetWriter<string[]>(field, (f, w, o) => w.WriteStringArray(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadStringArray(f));
+            }
+            else if (elemType == typeof(Guid?))
+            {
+                writeAction = GetWriter<Guid?[]>(field, (f, w, o) => w.WriteGuidArray(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadGuidArray(f));
+            } 
+            else if (elemType == typeof(DateTime?))
+            {
+                writeAction = GetWriter<DateTime?[]>(field, (f, w, o) => w.WriteDateArray(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadDateArray(f));
+            }
+            else if (elemType.IsEnum)
+            {
+                writeAction = GetWriter(field, MthdWriteEnumArray, elemType);
+                readAction = GetReader(field, MthdReadEnumArray, elemType);
+            }
+            else
+            {
+                writeAction = GetWriter(field, MthdWriteObjArray, elemType);
+                readAction = GetReader(field, MthdReadObjArray, elemType);
+            }  
+        }
+
+        /// <summary>
+        /// Handle other type.
+        /// </summary>
+        /// <param name="field">The field.</param>
+        /// <param name="writeAction">Write action.</param>
+        /// <param name="readAction">Read action.</param>
+        private static void HandleOther(FieldInfo field, out PortableReflectiveWriteAction writeAction,
+            out PortableReflectiveReadAction readAction)
+        {
+            var type = field.FieldType;
+
+            var genericDef = type.IsGenericType ? type.GetGenericTypeDefinition() : null;
+
+            bool nullable = genericDef == typeof(Nullable<>);
+
+            var nullableType = nullable ? type.GetGenericArguments()[0] : null;
+
+            if (type == typeof(decimal))
+            {
+                writeAction = GetWriter<decimal>(field, (f, w, o) => w.WriteDecimal(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadDecimal(f));
+            }
+            else if (type == typeof(string))
+            {
+                writeAction = GetWriter<string>(field, (f, w, o) => w.WriteString(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadString(f));
+            }
+            else if (type == typeof(Guid))
+            {
+                writeAction = GetWriter<Guid>(field, (f, w, o) => w.WriteGuid(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadGuid(f) ?? default(Guid));
+            }
+            else if (nullable && nullableType == typeof(Guid))
+            {
+                writeAction = GetWriter<Guid?>(field, (f, w, o) => w.WriteGuid(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadGuid(f));
+            } 
+            else if (type == typeof(DateTime))
+            {
+                writeAction = GetWriter<DateTime>(field, (f, w, o) => w.WriteDate(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadDate(f) ?? default(DateTime));
+            }
+            else if (nullable && nullableType == typeof(DateTime))
+            {
+                writeAction = GetWriter<DateTime?>(field, (f, w, o) => w.WriteDate(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadDate(f));
+            }
+            else if (type.IsEnum)
+            {
+                writeAction = GetWriter<object>(field, (f, w, o) => w.WriteEnum(f, o), true);
+                readAction = GetReader(field, MthdReadEnum);
+            }
+            else if (genericDef == PortableUtils.TypGenericDictionary ||
+                type.GetInterface(PortableUtils.TypGenericDictionary.FullName) != null)
+            {
+                writeAction = GetWriter(field, MthdWriteGenericDictionary, type.GetGenericArguments());
+                readAction = GetReader(field, MthdReadGenericDictionary, type.GetGenericArguments());
+            }
+            else if (genericDef == PortableUtils.TypGenericCollection ||
+                type.GetInterface(PortableUtils.TypGenericCollection.FullName) != null)
+            {
+                writeAction = GetWriter(field, MthdWriteGenericCollection, type.GetGenericArguments());
+                readAction = GetReader(field, MthdReadGenericCollection, type.GetGenericArguments());
+            }
+            else if (type == PortableUtils.TypDictionary || type.GetInterface(PortableUtils.TypDictionary.FullName) != null)
+            {
+                writeAction = GetWriter<IDictionary>(field, (f, w, o) => w.WriteDictionary(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadDictionary(f));
+            }
+            else if (type == PortableUtils.TypCollection || type.GetInterface(PortableUtils.TypCollection.FullName) != null)
+            {
+                writeAction = GetWriter<ICollection>(field, (f, w, o) => w.WriteCollection(f, o));
+                readAction = GetReader(field, (f, r) => r.ReadCollection(f));
+            }
+            else
+            {
+                writeAction = GetWriter(field, MthdWriteObj);
+                readAction = GetReader(field, MthdReadObj);
+            }                
+        }
+
+        /// <summary>
+        /// Gets the reader with a specified write action.
+        /// </summary>
+        private static PortableReflectiveWriteAction GetWriter<T>(FieldInfo field,
+            Expression<Action<string, IPortableWriter, T>> write,
+            bool convertFieldValToObject = false)
+        {
+            Debug.Assert(field != null);
+            Debug.Assert(field.DeclaringType != null);   // non-static
+
+            // Get field value
+            var targetParam = Expression.Parameter(typeof(object));
+            var targetParamConverted = Expression.Convert(targetParam, field.DeclaringType);
+            Expression fldExpr = Expression.Field(targetParamConverted, field);
+
+            if (convertFieldValToObject)
+                fldExpr = Expression.Convert(fldExpr, typeof (object));
+
+            // Call IPortableWriter method
+            var writerParam = Expression.Parameter(typeof(IPortableWriter));
+            var fldNameParam = Expression.Constant(PortableUtils.CleanFieldName(field.Name));
+            var writeExpr = Expression.Invoke(write, fldNameParam, writerParam, fldExpr);
+
+            // Compile and return
+            return Expression.Lambda<PortableReflectiveWriteAction>(writeExpr, targetParam, writerParam).Compile();
+        }
+
+        /// <summary>
+        /// Gets the writer with a specified generic method.
+        /// </summary>
+        private static PortableReflectiveWriteAction GetWriter(FieldInfo field, MethodInfo method, 
+            params Type[] genericArgs)
+        {
+            Debug.Assert(field != null);
+            Debug.Assert(field.DeclaringType != null);   // non-static
+
+            if (genericArgs.Length == 0)
+                genericArgs = new[] {field.FieldType};
+
+            // Get field value
+            var targetParam = Expression.Parameter(typeof(object));
+            var targetParamConverted = Expression.Convert(targetParam, field.DeclaringType);
+            var fldExpr = Expression.Field(targetParamConverted, field);
+
+            // Call IPortableWriter method
+            var writerParam = Expression.Parameter(typeof(IPortableWriter));
+            var fldNameParam = Expression.Constant(PortableUtils.CleanFieldName(field.Name));
+            var writeMethod = method.MakeGenericMethod(genericArgs);
+            var writeExpr = Expression.Call(writerParam, writeMethod, fldNameParam, fldExpr);
+
+            // Compile and return
+            return Expression.Lambda<PortableReflectiveWriteAction>(writeExpr, targetParam, writerParam).Compile();
+        }
+
+        /// <summary>
+        /// Gets the reader with a specified read action.
+        /// </summary>
+        private static PortableReflectiveReadAction GetReader<T>(FieldInfo field, 
+            Expression<Func<string, IPortableReader, T>> read)
+        {
+            Debug.Assert(field != null);
+            Debug.Assert(field.DeclaringType != null);   // non-static
+
+            // Call IPortableReader method
+            var readerParam = Expression.Parameter(typeof(IPortableReader));
+            var fldNameParam = Expression.Constant(PortableUtils.CleanFieldName(field.Name));
+            Expression readExpr = Expression.Invoke(read, fldNameParam, readerParam);
+
+            if (typeof(T) != field.FieldType)
+                readExpr = Expression.Convert(readExpr, field.FieldType);
+
+            // Assign field value
+            var targetParam = Expression.Parameter(typeof(object));
+            var targetParamConverted = Expression.Convert(targetParam, field.DeclaringType);
+            var assignExpr = Expression.Call(DelegateConverter.GetWriteFieldMethod(field), targetParamConverted, 
+                readExpr);
+
+            // Compile and return
+            return Expression.Lambda<PortableReflectiveReadAction>(assignExpr, targetParam, readerParam).Compile();
+        }
+
+        /// <summary>
+        /// Gets the reader with a specified generic method.
+        /// </summary>
+        private static PortableReflectiveReadAction GetReader(FieldInfo field, MethodInfo method, 
+            params Type[] genericArgs)
+        {
+            Debug.Assert(field != null);
+            Debug.Assert(field.DeclaringType != null);   // non-static
+
+            if (genericArgs.Length == 0)
+                genericArgs = new[] {field.FieldType};
+
+            // Call IPortableReader method
+            var readerParam = Expression.Parameter(typeof (IPortableReader));
+            var fldNameParam = Expression.Constant(PortableUtils.CleanFieldName(field.Name));
+            var readMethod = method.MakeGenericMethod(genericArgs);
+            Expression readExpr = Expression.Call(readerParam, readMethod, fldNameParam);
+
+            if (readMethod.ReturnType != field.FieldType)
+                readExpr = Expression.Convert(readExpr, field.FieldType);
+
+            // Assign field value
+            var targetParam = Expression.Parameter(typeof(object));
+            var targetParamConverted = Expression.Convert(targetParam, field.DeclaringType);
+            var assignExpr = Expression.Call(DelegateConverter.GetWriteFieldMethod(field), targetParamConverted, 
+                readExpr);
+
+            // Compile and return
+            return Expression.Lambda<PortableReflectiveReadAction>(assignExpr, targetParam, readerParam).Compile();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableReflectiveSerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableReflectiveSerializer.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableReflectiveSerializer.cs
new file mode 100644
index 0000000..3dff691
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableReflectiveSerializer.cs
@@ -0,0 +1,218 @@
+/*
+ * 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.Portable
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Reflection;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Portable serializer which reflectively writes all fields except of ones with 
+    /// <see cref="System.NonSerializedAttribute"/>.
+    /// <para />
+    /// Note that Java platform stores dates as a difference between current time 
+    /// and predefined absolute UTC date. Therefore, this difference is always the 
+    /// same for all time zones. .Net, in contrast, stores dates as a difference 
+    /// between current time and some predefined date relative to the current time 
+    /// zone. It means that this difference will be different as you change time zones. 
+    /// To overcome this discrepancy Ignite always converts .Net date to UTC form 
+    /// before serializing and allows user to decide whether to deserialize them 
+    /// in UTC or local form using <c>ReadDate(..., true/false)</c> methods in 
+    /// <see cref="IPortableReader"/> and <see cref="IPortableRawReader"/>.
+    /// This serializer always read dates in UTC form. It means that if you have
+    /// local date in any field/property, it will be implicitly converted to UTC
+    /// form after the first serialization-deserialization cycle. 
+    /// </summary>
+    internal class PortableReflectiveSerializer : IPortableSerializer
+    {
+        /** Cached binding flags. */
+        private static readonly BindingFlags Flags = BindingFlags.Instance | BindingFlags.Public |
+            BindingFlags.NonPublic | BindingFlags.DeclaredOnly;
+
+        /** Cached type descriptors. */
+        private readonly IDictionary<Type, Descriptor> _types = new Dictionary<Type, Descriptor>();
+
+        /// <summary>
+        /// Write portalbe object.
+        /// </summary>
+        /// <param name="obj">Object.</param>
+        /// <param name="writer">Portable writer.</param>
+        /// <exception cref="PortableException">Type is not registered in serializer:  + type.Name</exception>
+        public void WritePortable(object obj, IPortableWriter writer)
+        {
+            var portableMarshalAware = obj as IPortableMarshalAware;
+
+            if (portableMarshalAware != null)
+                portableMarshalAware.WritePortable(writer);
+            else
+                GetDescriptor(obj).Write(obj, writer);
+        }
+
+        /// <summary>
+        /// Read portable object.
+        /// </summary>
+        /// <param name="obj">Instantiated empty object.</param>
+        /// <param name="reader">Portable reader.</param>
+        /// <exception cref="PortableException">Type is not registered in serializer:  + type.Name</exception>
+        public void ReadPortable(object obj, IPortableReader reader)
+        {
+            var portableMarshalAware = obj as IPortableMarshalAware;
+            
+            if (portableMarshalAware != null)
+                portableMarshalAware.ReadPortable(reader);
+            else
+                GetDescriptor(obj).Read(obj, reader);
+        }
+
+        /// <summary>Register type.</summary>
+        /// <param name="type">Type.</param>
+        /// <param name="typeId">Type ID.</param>
+        /// <param name="converter">Name converter.</param>
+        /// <param name="idMapper">ID mapper.</param>
+        public void Register(Type type, int typeId, IPortableNameMapper converter,
+            IPortableIdMapper idMapper)
+        {
+            if (type.GetInterface(typeof(IPortableMarshalAware).Name) != null)
+                return;
+
+            List<FieldInfo> fields = new List<FieldInfo>();
+
+            Type curType = type;
+
+            while (curType != null)
+            {
+                foreach (FieldInfo field in curType.GetFields(Flags))
+                {
+                    if (!field.IsNotSerialized)
+                        fields.Add(field);
+                }
+
+                curType = curType.BaseType;
+            }
+
+            IDictionary<int, string> idMap = new Dictionary<int, string>();
+
+            foreach (FieldInfo field in fields)
+            {
+                string fieldName = PortableUtils.CleanFieldName(field.Name);
+
+                int fieldId = PortableUtils.FieldId(typeId, fieldName, converter, idMapper);
+
+                if (idMap.ContainsKey(fieldId))
+                {
+                    throw new PortableException("Conflicting field IDs [type=" +
+                        type.Name + ", field1=" + idMap[fieldId] + ", field2=" + fieldName +
+                        ", fieldId=" + fieldId + ']');
+                }
+                
+                idMap[fieldId] = fieldName;
+            }
+
+            fields.Sort(Compare);
+
+            Descriptor desc = new Descriptor(fields);
+
+            _types[type] = desc;
+        }
+
+        /// <summary>
+        /// Gets the descriptor for an object.
+        /// </summary>
+        private Descriptor GetDescriptor(object obj)
+        {
+            var type = obj.GetType();
+
+            Descriptor desc;
+
+            if (!_types.TryGetValue(type, out desc))
+                throw new PortableException("Type is not registered in serializer: " + type.Name);
+
+            return desc;
+        }
+        
+        /// <summary>
+        /// Compare two FieldInfo instances. 
+        /// </summary>
+        private static int Compare(FieldInfo info1, FieldInfo info2) {
+            string name1 = PortableUtils.CleanFieldName(info1.Name);
+            string name2 = PortableUtils.CleanFieldName(info2.Name);
+
+            return string.Compare(name1, name2, StringComparison.OrdinalIgnoreCase);
+        }
+
+        /// <summary>
+        /// Type descriptor. 
+        /// </summary>
+        private class Descriptor
+        {
+            /** Write actions to be performed. */
+            private readonly List<PortableReflectiveWriteAction> _wActions;
+
+            /** Read actions to be performed. */
+            private readonly List<PortableReflectiveReadAction> _rActions;
+
+            /// <summary>
+            /// Constructor.
+            /// </summary>
+            /// <param name="fields">Fields.</param>
+            public Descriptor(List<FieldInfo> fields)
+            {
+                _wActions = new List<PortableReflectiveWriteAction>(fields.Count);
+                _rActions = new List<PortableReflectiveReadAction>(fields.Count);
+
+                foreach (FieldInfo field in fields)
+                {
+                    PortableReflectiveWriteAction writeAction;
+                    PortableReflectiveReadAction readAction;
+
+                    PortableReflectiveActions.TypeActions(field, out writeAction, out readAction);
+
+                    _wActions.Add(writeAction);
+                    _rActions.Add(readAction);
+                }
+            }
+
+            /// <summary>
+            /// Write object.
+            /// </summary>
+            /// <param name="obj">Object.</param>
+            /// <param name="writer">Portable writer.</param>
+            public void Write(object obj, IPortableWriter writer)
+            {
+                int cnt = _wActions.Count;
+
+                for (int i = 0; i < cnt; i++)
+                    _wActions[i](obj, writer);                   
+            }
+
+            /// <summary>
+            /// Read object.
+            /// </summary>
+            /// <param name="obj">Object.</param>
+            /// <param name="reader">Portable reader.</param>
+            public void Read(object obj, IPortableReader reader)
+            {
+                int cnt = _rActions.Count;
+
+                for (int i = 0; i < cnt; i++ )
+                    _rActions[i](obj, reader);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableSurrogateTypeDescriptor.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableSurrogateTypeDescriptor.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableSurrogateTypeDescriptor.cs
new file mode 100644
index 0000000..c8dcc5a
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableSurrogateTypeDescriptor.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.Portable
+{
+    using System;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Surrogate type descriptor. Used in cases when type if identified by name and is not provided in configuration.
+    /// </summary>
+    internal class PortableSurrogateTypeDescriptor : IPortableTypeDescriptor
+    {
+        /** Portable configuration. */
+        private readonly PortableConfiguration _cfg;
+
+        /** Type ID. */
+        private readonly int _id;
+
+        /** Type name. */
+        private readonly string _name;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="cfg">Portable configuration.</param>
+        /// <param name="id">Type ID.</param>
+        public PortableSurrogateTypeDescriptor(PortableConfiguration cfg, int id)
+        {
+            _cfg = cfg;
+            _id = id;
+        }
+
+        /// <summary>
+        /// Constrcutor.
+        /// </summary>
+        /// <param name="cfg">Portable configuration.</param>
+        /// <param name="name">Type name.</param>
+        public PortableSurrogateTypeDescriptor(PortableConfiguration cfg, string name)
+        {
+            _cfg = cfg;
+            _name = name;
+
+            _id = PortableUtils.TypeId(name, cfg.DefaultNameMapper, cfg.DefaultIdMapper);
+        }
+
+        /** <inheritDoc /> */
+        public Type Type
+        {
+            get { return null; }
+        }
+
+        /** <inheritDoc /> */
+        public int TypeId
+        {
+            get { return _id; }
+        }
+
+        /** <inheritDoc /> */
+        public string TypeName
+        {
+            get { return _name; }
+        }
+
+        /** <inheritDoc /> */
+        public bool UserType
+        {
+            get { return true; }
+        }
+
+        /** <inheritDoc /> */
+        public bool MetadataEnabled
+        {
+            get { return _cfg.DefaultMetadataEnabled; }
+        }
+
+        /** <inheritDoc /> */
+        public bool KeepDeserialized
+        {
+            get { return _cfg.DefaultKeepDeserialized; }
+        }
+
+        /** <inheritDoc /> */
+        public IPortableNameMapper NameConverter
+        {
+            get { return _cfg.DefaultNameMapper; }
+        }
+
+        /** <inheritDoc /> */
+        public IPortableIdMapper Mapper
+        {
+            get { return _cfg.DefaultIdMapper; }
+        }
+
+        /** <inheritDoc /> */
+        public IPortableSerializer Serializer
+        {
+            get { return _cfg.DefaultSerializer; }
+        }
+
+        /** <inheritDoc /> */
+        public string AffinityKeyFieldName
+        {
+            get { return null; }
+        }
+
+        /** <inheritDoc /> */
+        public object TypedHandler
+        {
+            get { return null; }
+        }
+
+        /** <inheritDoc /> */
+        public PortableSystemWriteDelegate UntypedHandler
+        {
+            get { return null; }
+        }
+    }
+}


[33/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/JobEvent.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/JobEvent.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/JobEvent.cs
new file mode 100644
index 0000000..81d537f
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/JobEvent.cs
@@ -0,0 +1,100 @@
+/*
+ * 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.Events
+{
+    using System;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Ignite job event.
+    /// </summary>
+    public sealed class JobEvent : EventBase
+	{
+        /** */
+        private readonly string _taskName;
+
+        /** */
+        private readonly string _taskClassName;
+
+        /** */
+        private readonly IgniteGuid _taskSessionId;
+
+        /** */
+        private readonly IgniteGuid _jobId;
+
+        /** */
+        private readonly IClusterNode _taskNode;
+
+        /** */
+        private readonly Guid _taskSubjectId;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="r">The reader to read data from.</param>
+        internal JobEvent(IPortableRawReader r) : base(r)
+        {
+            _taskName = r.ReadString();
+            _taskClassName = r.ReadString();
+            _taskSessionId = IgniteGuid.ReadPortable(r);
+            _jobId = IgniteGuid.ReadPortable(r);
+            _taskNode = ReadNode(r);
+            _taskSubjectId = r.ReadGuid() ?? Guid.Empty;
+        }
+		
+        /// <summary>
+        /// Gets name of the task that triggered the event. 
+        /// </summary>
+        public string TaskName { get { return _taskName; } }
+
+        /// <summary>
+        /// Gets name of task class that triggered this event. 
+        /// </summary>
+        public string TaskClassName { get { return _taskClassName; } }
+
+        /// <summary>
+        /// Gets task session ID of the task that triggered this event. 
+        /// </summary>
+        public IgniteGuid TaskSessionId { get { return _taskSessionId; } }
+
+        /// <summary>
+        /// Gets job ID. 
+        /// </summary>
+        public IgniteGuid JobId { get { return _jobId; } }
+
+        /// <summary>
+        /// Get node where parent task of the job has originated. 
+        /// </summary>
+        public IClusterNode TaskNode { get { return _taskNode; } }
+
+        /// <summary>
+        /// Gets task subject ID. 
+        /// </summary>
+        public Guid TaskSubjectId { get { return _taskSubjectId; } }
+
+        /** <inheritDoc /> */
+	    public override string ToShortString()
+	    {
+	        return string.Format("{0}: TaskName={1}, TaskClassName={2}, TaskSessionId={3}, JobId={4}, TaskNode={5}, " +
+	                             "TaskSubjectId={6}", Name, TaskName, TaskClassName, TaskSessionId, JobId, TaskNode, 
+                                 TaskSubjectId);
+	    }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/SwapSpaceEvent.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/SwapSpaceEvent.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/SwapSpaceEvent.cs
new file mode 100644
index 0000000..676c2e0
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/SwapSpaceEvent.cs
@@ -0,0 +1,50 @@
+/*
+ * 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.Events
+{
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Grid swap space event.
+    /// </summary>
+    public sealed class SwapSpaceEvent : EventBase
+	{
+        /** */
+        private readonly string _space;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="r">The reader to read data from.</param>
+        internal SwapSpaceEvent(IPortableRawReader r) : base(r)
+        {
+            _space = r.ReadString();
+        }
+		
+        /// <summary>
+        /// Gets swap space name. 
+        /// </summary>
+        public string Space { get { return _space; } }
+
+        /** <inheritDoc /> */
+	    public override string ToShortString()
+	    {
+	        return string.Format("{0}: Space={1}", Name, Space);
+	    }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/TaskEvent.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/TaskEvent.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/TaskEvent.cs
new file mode 100644
index 0000000..7149fb3
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Events/TaskEvent.cs
@@ -0,0 +1,91 @@
+/*
+ * 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.Events
+{
+    using System;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Ignite task event.
+    /// </summary>
+    public sealed class TaskEvent : EventBase
+	{
+        /** */
+        private readonly string _taskName;
+
+        /** */
+        private readonly string _taskClassName;
+
+        /** */
+        private readonly IgniteGuid _taskSessionId;
+
+        /** */
+        private readonly bool _internal;
+
+        /** */
+        private readonly Guid _subjectId;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="r">The reader to read data from.</param>
+        internal TaskEvent(IPortableRawReader r) : base(r)
+        {
+            _taskName = r.ReadString();
+            _taskClassName = r.ReadString();
+            _taskSessionId = IgniteGuid.ReadPortable(r);
+            _internal = r.ReadBoolean();
+            _subjectId = r.ReadGuid() ?? Guid.Empty;
+        }
+		
+        /// <summary>
+        /// Gets name of the task that triggered the event. 
+        /// </summary>
+        public string TaskName { get { return _taskName; } }
+
+        /// <summary>
+        /// Gets name of task class that triggered this event. 
+        /// </summary>
+        public string TaskClassName { get { return _taskClassName; } }
+
+        /// <summary>
+        /// Gets session ID of the task that triggered the event. 
+        /// </summary>
+        public IgniteGuid TaskSessionId { get { return _taskSessionId; } }
+
+        /// <summary>
+        /// Returns true if task is created by Ignite and is used for system needs. 
+        /// </summary>
+        public bool Internal { get { return _internal; } }
+
+        /// <summary>
+        /// Gets security subject ID initiated this task event, if available. This property is not available for 
+        /// <see cref="EventType.EvtTaskSessionAttrSet" /> task event. 
+        /// Subject ID will be set either to node ID or client ID initiated task execution. 
+        /// </summary>
+        public Guid SubjectId { get { return _subjectId; } }
+
+        /** <inheritDoc /> */
+	    public override string ToShortString()
+	    {
+	        return string.Format("{0}: TaskName={1}, TaskClassName={2}, TaskSessionId={3}, Internal={4}, " +
+	                             "SubjectId={5}", Name, TaskName, TaskClassName, TaskSessionId, Internal, SubjectId);
+	    }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/IIgnite.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/IIgnite.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/IIgnite.cs
new file mode 100644
index 0000000..b691254
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/IIgnite.cs
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core
+{
+    using System;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Datastream;
+    using Apache.Ignite.Core.Events;
+    using Apache.Ignite.Core.Messaging;
+    using Apache.Ignite.Core.Portable;
+    using Apache.Ignite.Core.Services;
+    using Apache.Ignite.Core.Transactions;
+
+    /// <summary>
+    /// Main entry point for all Ignite APIs.
+    /// You can obtain an instance of <c>IGrid</c> through <see cref="Ignition.GetIgnite()"/>,
+    /// or for named grids you can use <see cref="Ignition.GetIgnite(string)"/>. Note that you
+    /// can have multiple instances of <c>IGrid</c> running in the same process by giving
+    /// each instance a different name.
+    /// <para/>
+    /// All members are thread-safe and may be used concurrently from multiple threads.
+    /// </summary>
+    public interface IIgnite : IDisposable
+    {
+        /// <summary>
+        /// Gets the name of the grid this Ignite instance (and correspondingly its local node) belongs to.
+        /// Note that single process can have multiple Ignite instances all belonging to different grids. Grid
+        /// name allows to indicate to what grid this particular Ignite instance (i.e. Ignite runtime and its
+        /// local node) belongs to.
+        /// <p/>
+        /// If default Ignite instance is used, then <c>null</c> is returned. Refer to <see cref="Ignition"/> documentation
+        /// for information on how to start named grids.
+        /// </summary>
+        /// <returns>Name of the grid, or <c>null</c> for default grid.</returns>
+        string Name { get; }
+
+        /// <summary>
+        /// Gets an instance of <see cref="ICluster" /> interface.
+        /// </summary>
+        ICluster Cluster { get; }
+
+        /// <summary>
+        /// Gets compute functionality over this grid projection. All operations
+        /// on the returned ICompute instance will only include nodes from
+        /// this projection.
+        /// </summary>
+        /// <returns>Compute instance over this grid projection.</returns>
+        ICompute Compute();
+
+        /// <summary>
+        /// Gets compute functionality over specified grid projection. All operations
+        /// on the returned ICompute instance will only include nodes from
+        /// that projection.
+        /// </summary>
+        /// <returns>Compute instance over specified grid projection.</returns>
+        ICompute Compute(IClusterGroup clusterGroup);
+
+        /// <summary>
+        /// Gets the cache instance for the given name to work with keys and values of specified types.
+        /// <para/>
+        /// You can get instances of ICache of the same name, but with different key/value types.
+        /// These will use the same named cache, but only allow working with entries of specified types.
+        /// Attempt to retrieve an entry of incompatible type will result in <see cref="InvalidCastException"/>.
+        /// Use <see cref="Cache{Object, Object}"/> in order to work with entries of arbitrary types.
+        /// </summary>
+        /// <param name="name">Cache name.</param>
+        /// <returns>Cache instance for given name.</returns>
+        /// <typeparam name="TK">Cache key type.</typeparam>
+        /// <typeparam name="TV">Cache value type.</typeparam>
+        ICache<TK, TV> Cache<TK, TV>(string name);
+
+        /// <summary>
+        /// Gets existing cache with the given name or creates new one using template configuration.
+        /// </summary>
+        /// <typeparam name="TK">Cache key type.</typeparam>
+        /// <typeparam name="TV">Cache value type.</typeparam>
+        /// <param name="name">Cache name.</param>
+        /// <returns>Existing or newly created cache.</returns>
+        ICache<TK, TV> GetOrCreateCache<TK, TV>(string name);
+
+        /// <summary>
+        /// Dynamically starts new cache using template configuration.
+        /// </summary>
+        /// <typeparam name="TK">Cache key type.</typeparam>
+        /// <typeparam name="TV">Cache value type.</typeparam>
+        /// <param name="name">Cache name.</param>
+        /// <returns>Existing or newly created cache.</returns>
+        ICache<TK, TV> CreateCache<TK, TV>(string name);
+
+        /// <summary>
+        /// Gets a new instance of data streamer associated with given cache name. Data streamer
+        /// is responsible for loading external data into Ignite. For more information
+        /// refer to <see cref="IDataStreamer{K,V}"/> documentation.
+        /// </summary>
+        /// <param name="cacheName">Cache name (<c>null</c> for default cache).</param>
+        /// <returns>Data streamer.</returns>
+        IDataStreamer<TK, TV> DataStreamer<TK, TV>(string cacheName);
+
+        /// <summary>
+        /// Gets an instance of <see cref="IPortables"/> interface.
+        /// </summary>
+        /// <returns>Instance of <see cref="IPortables"/> interface</returns>
+        IPortables Portables();
+
+        /// <summary>
+        /// Gets affinity service to provide information about data partitioning and distribution.
+        /// </summary>
+        /// <param name="name">Cache name.</param>
+        /// <returns>Cache data affinity service.</returns>
+        ICacheAffinity Affinity(string name);
+
+        /// <summary>
+        /// Gets  Ignite transactions facade.
+        /// </summary>
+        ITransactions Transactions { get; }
+
+        /// <summary>
+        /// Gets messaging facade over all cluster nodes.
+        /// </summary>
+        /// <returns>Messaging instance over all cluster nodes.</returns>
+        IMessaging Message();
+
+        /// <summary>
+        /// Gets messaging facade over nodes within the cluster group.  All operations on the returned 
+        /// <see cref="IMessaging"/>> instance will only include nodes from the specified cluster group.
+        /// </summary>
+        /// <param name="clusterGroup">Cluster group.</param>
+        /// <returns>Messaging instance over given cluster group.</returns>
+        IMessaging Message(IClusterGroup clusterGroup);
+
+        /// <summary>
+        /// Gets events facade over all cluster nodes.
+        /// </summary>
+        /// <returns>Events facade over all cluster nodes.</returns>
+        IEvents Events();
+
+        /// <summary>
+        /// Gets events facade over nodes within the cluster group.  All operations on the returned 
+        /// <see cref="IEvents"/>> instance will only include nodes from the specified cluster group.
+        /// </summary>
+        /// <param name="clusterGroup">Cluster group.</param>
+        /// <returns>Events instance over given cluster group.</returns>
+        IEvents Events(IClusterGroup clusterGroup);
+
+        /// <summary>
+        /// Gets services facade over all cluster nodes.
+        /// </summary>
+        /// <returns>Services facade over all cluster nodes.</returns>
+        IServices Services();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
new file mode 100644
index 0000000..5a03e93
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
@@ -0,0 +1,140 @@
+/*
+ * 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.Collections.Generic;
+    using System.Diagnostics.CodeAnalysis;
+    using Apache.Ignite.Core.Lifecycle;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Grid configuration.
+    /// </summary>
+    public class IgniteConfiguration
+    {
+        /// <summary>
+        /// Default initial JVM memory in megabytes.
+        /// </summary>
+        public const int DefaultJvmInitMem = 512;
+
+        /// <summary>
+        /// Default maximum JVM memory in megabytes.
+        /// </summary>
+        public const int DefaultJvmMaxMem = 1024;
+
+        /// <summary>
+        /// Default constructor.
+        /// </summary>
+        public IgniteConfiguration()
+        {
+            JvmInitialMemoryMb = DefaultJvmInitMem;
+            JvmMaxMemoryMb = DefaultJvmMaxMem;
+        }
+
+        /// <summary>
+        /// Copying constructor.
+        /// </summary>
+        /// <param name="cfg">Configuration.</param>
+        internal IgniteConfiguration(IgniteConfiguration cfg)
+        {
+            SpringConfigUrl = cfg.SpringConfigUrl;
+            JvmDllPath = cfg.JvmDllPath;
+            IgniteHome = cfg.IgniteHome;
+            JvmClasspath = cfg.JvmClasspath;
+            SuppressWarnings = cfg.SuppressWarnings;
+
+            JvmOptions = cfg.JvmOptions != null ? new List<string>(cfg.JvmOptions) : null;
+            Assemblies = cfg.Assemblies != null ? new List<string>(cfg.Assemblies) : null;
+
+            PortableConfiguration = cfg.PortableConfiguration != null
+                ? new PortableConfiguration(cfg.PortableConfiguration)
+                : null;
+
+            LifecycleBeans = cfg.LifecycleBeans != null ? new List<ILifecycleBean>(cfg.LifecycleBeans) : null;
+
+            JvmInitialMemoryMb = cfg.JvmInitialMemoryMb;
+            JvmMaxMemoryMb = cfg.JvmMaxMemoryMb;
+        }
+
+        /// <summary>
+        /// Gets or sets the portable configuration.
+        /// </summary>
+        /// <value>
+        /// The portable configuration.
+        /// </value>
+        public PortableConfiguration PortableConfiguration { get; set; }
+
+        /// <summary>
+        /// URL to Spring configuration file.
+        /// </summary>
+        [SuppressMessage("Microsoft.Design", "CA1056:UriPropertiesShouldNotBeStrings")]
+        public string SpringConfigUrl { get; set; }
+
+        /// <summary>
+        /// Path jvm.dll file. If not set, it's location will be determined
+        /// using JAVA_HOME environment variable.
+        /// If path is neither set nor determined automatically, an exception
+        /// will be thrown.
+        /// </summary>
+        public string JvmDllPath { get; set; }
+
+        /// <summary>
+        /// Path to Ignite home. If not set environment variable IGNITE_HOME will be used.
+        /// </summary>
+        public string IgniteHome { get; set; }
+
+        /// <summary>
+        /// Classpath used by JVM on Ignite start.
+        /// </summary>
+        public string JvmClasspath { get; set; }
+
+        /// <summary>
+        /// Collection of options passed to JVM on Ignite start.
+        /// </summary>
+        public ICollection<string> JvmOptions { get; set; }
+
+        /// <summary>
+        /// List of additional .Net assemblies to load on Ignite start. Each item can be either
+        /// fully qualified assembly name, path to assembly to DLL or path to a directory when 
+        /// assemblies reside.
+        /// </summary>
+        public IList<string> Assemblies { get; set; }
+
+        /// <summary>
+        /// Whether to suppress warnings.
+        /// </summary>
+        public bool SuppressWarnings { get; set; }
+
+        /// <summary>
+        /// Lifecycle beans.
+        /// </summary>
+        public ICollection<ILifecycleBean> LifecycleBeans { get; set; }
+
+        /// <summary>
+        /// Initial amount of memory in megabytes given to JVM. Maps to -Xms Java option.
+        /// Defaults to <see cref="DefaultJvmInitMem"/>.
+        /// </summary>
+        public int JvmInitialMemoryMb { get; set; }
+
+        /// <summary>
+        /// Maximum amount of memory in megabytes given to JVM. Maps to -Xmx Java option.
+        /// Defaults to <see cref="DefaultJvmMaxMem"/>.
+        /// </summary>
+        public int JvmMaxMemoryMb { get; set; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Ignition.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Ignition.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Ignition.cs
index a61a1a8..ef79008 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Ignition.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Ignition.cs
@@ -1,4 +1,4 @@
-/*
+/*
  * 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.
@@ -15,9 +15,629 @@
  * limitations under the License.
  */
 
-namespace Apache.Ignite.Core
+namespace Apache.Ignite.Core 
 {
-    public class Ignition
+    using System;
+    using System.Collections.Generic;
+    using System.IO;
+    using System.Linq;
+    using System.Reflection;
+    using System.Runtime;
+    using System.Runtime.InteropServices;
+    using System.Threading;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Handle;
+    using Apache.Ignite.Core.Impl.Interop;
+    using Apache.Ignite.Core.Impl.Memory;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+    using Apache.Ignite.Core.Lifecycle;
+    using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils;
+    using PU = Apache.Ignite.Core.Impl.Portable.PortableUtils;
+    
+    /// <summary>
+    /// This class defines a factory for the main Ignite API.
+    /// <p/>
+    /// Use <see cref="Ignition.Start()"/> method to start Ignite with default configuration.
+    /// <para/>
+    /// All members are thread-safe and may be used concurrently from multiple threads.
+    /// <example>
+    /// You can also use <see cref="IgniteConfiguration"/> to override some default configuration.
+    /// Below is an example on how to start Ignite with custom configuration for portable types and
+    /// provide path to Spring XML configuration file:
+    /// <code>
+    /// IgniteConfiguration cfg = new IgniteConfiguration();
+    ///
+    /// // Create portable type configuration.
+    /// PortableConfiguration portableCfg = new PortableConfiguration();
+    ///
+    /// cfg.SpringConfigUrl = "examples\\config\\example-cache.xml";
+    ///
+    /// portableCfg.TypeConfigurations = new List&lt;PortableTypeConfiguration&gt; 
+    /// {
+    ///     new PortableTypeConfiguration(typeof(Address)),
+    ///     new PortableTypeConfiguration(typeof(Organization))
+    /// };
+    ///
+    /// cfg.PortableConfiguration = portableCfg;
+    ///
+    /// // Start Ignite node with Ignite configuration.
+    /// var ignite = Ignition.Start(cfg);
+    /// </code>
+    /// </example>
+    /// </summary>
+    public static class Ignition
     {
+        /** */
+        private const string DefaultCfg = "config/default-config.xml";
+
+        /** */
+        private static readonly object SyncRoot = new object();
+
+        /** GC warning flag. */
+        private static int _gcWarn;
+
+        /** */
+        private static readonly IDictionary<NodeKey, Ignite> Nodes = new Dictionary<NodeKey, Ignite>();
+        
+        /** Current DLL name. */
+        private static readonly string IgniteDllName = Path.GetFileName(Assembly.GetExecutingAssembly().Location);
+
+        /** Startup info. */
+        [ThreadStatic]
+        private static Startup _startup;
+
+        /** Client mode flag. */
+        [ThreadStatic]
+        private static bool _clientMode;
+
+        /// <summary>
+        /// Static initializer.
+        /// </summary>
+        static Ignition()
+        {
+            AppDomain.CurrentDomain.AssemblyResolve += CurrentDomain_AssemblyResolve;
+        }
+
+        /// <summary>
+        /// Gets or sets a value indicating whether Ignite should be started in client mode.
+        /// Client nodes cannot hold data in caches.
+        /// </summary>
+        public static bool ClientMode
+        {
+            get { return _clientMode; }
+            set { _clientMode = value; }
+        }
+
+        /// <summary>
+        /// Starts Ignite with default configuration. By default this method will
+        /// use Ignite configuration defined in <code>IGNITE/config/default-config.xml</code>
+        /// configuration file. If such file is not found, then all system defaults will be used.
+        /// </summary>
+        /// <returns>Started Ignite.</returns>
+        public static IIgnite Start()
+        {
+            return Start(new IgniteConfiguration());
+        }
+
+        /// <summary>
+        /// Starts all grids specified within given Spring XML configuration file. If Ignite with given name
+        /// is already started, then exception is thrown. In this case all instances that may
+        /// have been started so far will be stopped too.
+        /// </summary>
+        /// <param name="springCfgPath">Spring XML configuration file path or URL. Note, that the path can be
+        /// absolute or relative to IGNITE_HOME.</param>
+        /// <returns>Started Ignite. If Spring configuration contains multiple Ignite instances, then the 1st
+        /// found instance is returned.</returns>
+        public static IIgnite Start(string springCfgPath)
+        {
+            return Start(new IgniteConfiguration {SpringConfigUrl = springCfgPath});
+        }
+
+        /// <summary>
+        /// Starts Ignite with given configuration.
+        /// </summary>
+        /// <returns>Started Ignite.</returns>
+        public unsafe static IIgnite Start(IgniteConfiguration cfg)
+        {
+            IgniteArgumentCheck.NotNull(cfg, "cfg");
+
+            // Copy configuration to avoid changes to user-provided instance.
+            IgniteConfigurationEx cfgEx = cfg as IgniteConfigurationEx;
+
+            cfg = cfgEx == null ? new IgniteConfiguration(cfg) : new IgniteConfigurationEx(cfgEx);
+
+            // Set default Spring config if needed.
+            if (cfg.SpringConfigUrl == null)
+                cfg.SpringConfigUrl = DefaultCfg;
+
+            lock (SyncRoot)
+            {
+                // 1. Check GC settings.
+                CheckServerGc(cfg);
+
+                // 2. Create context.
+                IgniteUtils.LoadDlls(cfg.JvmDllPath);
+
+                var cbs = new UnmanagedCallbacks();
+
+                void* ctx = IgniteManager.GetContext(cfg, cbs);
+
+                sbyte* cfgPath0 = IgniteUtils.StringToUtf8Unmanaged(cfg.SpringConfigUrl ?? DefaultCfg);
+
+                string gridName = cfgEx != null ? cfgEx.GridName : null;
+                sbyte* gridName0 = IgniteUtils.StringToUtf8Unmanaged(gridName);
+
+                // 3. Create startup object which will guide us through the rest of the process.
+                _startup = new Startup(cfg) { Context = ctx };
+
+                IUnmanagedTarget interopProc = null;
+
+                try
+                {
+                    // 4. Initiate Ignite start.
+                    interopProc = UU.IgnitionStart(cbs.Context, cfg.SpringConfigUrl ?? DefaultCfg, 
+                        cfgEx != null ? cfgEx.GridName : null, ClientMode);
+
+                    // 5. At this point start routine is finished. We expect STARTUP object to have all necessary data.
+                    Ignite node = new Ignite(cfg, _startup.Name, interopProc, _startup.Marshaller, 
+                        _startup.LifecycleBeans, cbs);
+
+                    // 6. On-start callback (notify lifecycle components).
+                    node.OnStart();
+
+                    Nodes[new NodeKey(_startup.Name)] = node;
+
+                    return node;
+                }
+                catch (Exception)
+                {
+                    // 1. Perform keys cleanup.
+                    string name = _startup.Name;
+
+                    if (name != null)
+                    {
+                        NodeKey key = new NodeKey(name);
+
+                        if (Nodes.ContainsKey(key))
+                            Nodes.Remove(key);
+                    }
+
+                    // 2. Stop Ignite node if it was started.
+                    if (interopProc != null)
+                        UU.IgnitionStop(interopProc.Context, gridName, true);
+
+                    // 3. Throw error further (use startup error if exists because it is more precise).
+                    if (_startup.Error != null)
+                        throw _startup.Error;
+
+                    throw;
+                }
+                finally
+                {
+                    _startup = null;
+
+                    Marshal.FreeHGlobal((IntPtr)cfgPath0);
+
+                    if ((IntPtr)gridName0 != IntPtr.Zero)
+                        Marshal.FreeHGlobal((IntPtr)gridName0);
+
+                    if (interopProc != null)
+                        UU.ProcessorReleaseStart(interopProc);
+                }
+            }
+        }
+
+        /// <summary>
+        /// Check whether GC is set to server mode.
+        /// </summary>
+        /// <param name="cfg">Configuration.</param>
+        private static void CheckServerGc(IgniteConfiguration cfg)
+        {
+            if (!cfg.SuppressWarnings && !GCSettings.IsServerGC && Interlocked.CompareExchange(ref _gcWarn, 1, 0) == 0)
+                Console.WriteLine("GC server mode is not enabled, this could lead to less " +
+                    "than optimal performance on multi-core machines (to enable see " +
+                    "http://msdn.microsoft.com/en-us/library/ms229357(v=vs.110).aspx).");
+        }
+
+        /// <summary>
+        /// Prepare callback invoked from Java.
+        /// </summary>
+        /// <param name="inStream">Intput stream with data.</param>
+        /// <param name="outStream">Output stream.</param>
+        /// <param name="handleRegistry">Handle registry.</param>
+        internal static void OnPrepare(PlatformMemoryStream inStream, PlatformMemoryStream outStream, 
+            HandleRegistry handleRegistry)
+        {
+            try
+            {
+                PortableReaderImpl reader = PU.Marshaller.StartUnmarshal(inStream);
+
+                PrepareConfiguration(reader.ReadObject<InteropDotNetConfiguration>());
+
+                PrepareLifecycleBeans(reader, outStream, handleRegistry);
+            }
+            catch (Exception e)
+            {
+                _startup.Error = e;
+
+                throw;
+            }
+        }
+
+        /// <summary>
+        /// Preapare configuration.
+        /// </summary>
+        /// <param name="dotNetCfg">Dot net configuration.</param>
+        private static void PrepareConfiguration(InteropDotNetConfiguration dotNetCfg)
+        {
+            // 1. Load assemblies.
+            IgniteConfiguration cfg = _startup.Configuration;
+
+            LoadAssemblies(cfg.Assemblies);
+
+            if (dotNetCfg != null)
+                LoadAssemblies(dotNetCfg.Assemblies);
+
+            // 2. Create marshaller only after assemblies are loaded.
+            if (cfg.PortableConfiguration == null && dotNetCfg != null && dotNetCfg.PortableCfg != null)
+                cfg.PortableConfiguration = dotNetCfg.PortableCfg.ToPortableConfiguration();
+
+            _startup.Marshaller = new PortableMarshaller(cfg.PortableConfiguration);
+        }
+
+        /// <summary>
+        /// Prepare lifecycle beans.
+        /// </summary>
+        /// <param name="reader">Reader.</param>
+        /// <param name="outStream">Output stream.</param>
+        /// <param name="handleRegistry">Handle registry.</param>
+        private static void PrepareLifecycleBeans(PortableReaderImpl reader, PlatformMemoryStream outStream, 
+            HandleRegistry handleRegistry)
+        {
+            IList<LifecycleBeanHolder> beans = new List<LifecycleBeanHolder>();
+
+            // 1. Read beans defined in Java.
+            int cnt = reader.ReadInt();
+
+            for (int i = 0; i < cnt; i++)
+                beans.Add(new LifecycleBeanHolder(CreateLifecycleBean(reader)));
+
+            // 2. Append beans definied in local configuration.
+            ICollection<ILifecycleBean> nativeBeans = _startup.Configuration.LifecycleBeans;
+
+            if (nativeBeans != null)
+            {
+                foreach (ILifecycleBean nativeBean in nativeBeans)
+                    beans.Add(new LifecycleBeanHolder(nativeBean));
+            }
+
+            // 3. Write bean pointers to Java stream.
+            outStream.WriteInt(beans.Count);
+
+            foreach (LifecycleBeanHolder bean in beans)
+                outStream.WriteLong(handleRegistry.AllocateCritical(bean));
+
+            outStream.SynchronizeOutput();
+
+            // 4. Set beans to STARTUP object.
+            _startup.LifecycleBeans = beans;
+        }
+
+        /// <summary>
+        /// Create lifecycle bean.
+        /// </summary>
+        /// <param name="reader">Reader.</param>
+        /// <returns>Lifecycle bean.</returns>
+        internal static ILifecycleBean CreateLifecycleBean(PortableReaderImpl reader)
+        {
+            // 1. Instantiate.
+            string assemblyName = reader.ReadString();
+            string clsName = reader.ReadString();
+
+            object bean = IgniteUtils.CreateInstance(assemblyName, clsName);
+
+            // 2. Set properties.
+            IDictionary<string, object> props = reader.ReadGenericDictionary<string, object>();
+
+            IgniteUtils.SetProperties(bean, props);
+
+            return bean as ILifecycleBean;
+        }
+
+        /// <summary>
+        /// Kernal start callback.
+        /// </summary>
+        /// <param name="stream">Stream.</param>
+        internal static void OnStart(IPortableStream stream)
+        {
+            try
+            {
+                // 1. Read data and leave critical state ASAP.
+                PortableReaderImpl reader = PU.Marshaller.StartUnmarshal(stream);
+                
+                // ReSharper disable once PossibleInvalidOperationException
+                var name = reader.ReadString();
+                
+                // 2. Set ID and name so that Start() method can use them later.
+                _startup.Name = name;
+
+                if (Nodes.ContainsKey(new NodeKey(name)))
+                    throw new IgniteException("Ignite with the same name already started: " + name);
+
+            }
+            catch (Exception e)
+            {
+                // 5. Preserve exception to throw it later in the "Start" method and throw it further
+                //    to abort startup in Java.
+                _startup.Error = e;
+
+                throw;
+            }
+        }
+
+        /// <summary>
+        /// Load assemblies.
+        /// </summary>
+        /// <param name="assemblies">Assemblies.</param>
+        private static void LoadAssemblies(IEnumerable<string> assemblies)
+        {
+            if (assemblies != null)
+            {
+                foreach (string s in assemblies)
+                {
+                    // 1. Try loading as directory.
+                    if (Directory.Exists(s))
+                    {
+                        string[] files = Directory.GetFiles(s, "*.dll");
+
+#pragma warning disable 0168
+
+                        foreach (string dllPath in files)
+                        {
+                            if (!SelfAssembly(dllPath))
+                            {
+                                try
+                                {
+                                    Assembly.LoadFile(dllPath);
+                                }
+
+                                catch (BadImageFormatException)
+                                {
+                                    // No-op.
+                                }
+                            }
+                        }
+
+#pragma warning restore 0168
+
+                        continue;
+                    }
+
+                    // 2. Try loading using full-name.
+                    try
+                    {
+                        Assembly assembly = Assembly.Load(s);
+
+                        if (assembly != null)
+                            continue;
+                    }
+                    catch (Exception e)
+                    {
+                        if (!(e is FileNotFoundException || e is FileLoadException))
+                            throw new IgniteException("Failed to load assembly: " + s, e);
+                    }
+
+                    // 3. Try loading using file path.
+                    try
+                    {
+                        Assembly assembly = Assembly.LoadFrom(s);
+
+                        if (assembly != null)
+                            continue;
+                    }
+                    catch (Exception e)
+                    {
+                        if (!(e is FileNotFoundException || e is FileLoadException))
+                            throw new IgniteException("Failed to load assembly: " + s, e);
+                    }
+
+                    // 4. Not found, exception.
+                    throw new IgniteException("Failed to load assembly: " + s);
+                }
+            }
+        }
+
+        /// <summary>
+        /// Whether assembly points to Ignite binary.
+        /// </summary>
+        /// <param name="assembly">Assembly to check..</param>
+        /// <returns><c>True</c> if this is one of GG assemblies.</returns>
+        private static bool SelfAssembly(string assembly)
+        {
+            return assembly.EndsWith(IgniteDllName, StringComparison.OrdinalIgnoreCase);
+        }
+
+        /// <summary>
+        /// Gets a named Ignite instance. If Ignite name is {@code null} or empty string,
+        /// then default no-name Ignite will be returned. Note that caller of this method
+        /// should not assume that it will return the same instance every time.
+        /// <p/>
+        /// Note that single process can run multiple Ignite instances and every Ignite instance (and its
+        /// node) can belong to a different grid. Ignite name defines what grid a particular Ignite
+        /// instance (and correspondingly its node) belongs to.
+        /// </summary>
+        /// <param name="name">Ignite name to which requested Ignite instance belongs. If <code>null</code>,
+        /// then Ignite instance belonging to a default no-name Ignite will be returned.
+        /// </param>
+        /// <returns>An instance of named grid.</returns>
+        public static IIgnite GetIgnite(string name)
+        {
+            lock (SyncRoot)
+            {
+                Ignite result;
+
+                if (!Nodes.TryGetValue(new NodeKey(name), out result))
+                    throw new IgniteException("Ignite instance was not properly started or was already stopped: " + name);
+
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Gets an instance of default no-name grid. Note that
+        /// caller of this method should not assume that it will return the same
+        /// instance every time.
+        /// </summary>
+        /// <returns>An instance of default no-name grid.</returns>
+        public static IIgnite GetIgnite()
+        {
+            return GetIgnite(null);
+        }
+
+        /// <summary>
+        /// Stops named grid. If <code>cancel</code> flag is set to <code>true</code> then
+        /// all jobs currently executing on local node will be interrupted. If
+        /// grid name is <code>null</code>, then default no-name Ignite will be stopped.
+        /// </summary>
+        /// <param name="name">Grid name. If <code>null</code>, then default no-name Ignite will be stopped.</param>
+        /// <param name="cancel">If <code>true</code> then all jobs currently executing will be cancelled
+        /// by calling <code>ComputeJob.cancel</code>method.</param>
+        /// <returns><code>true</code> if named Ignite instance was indeed found and stopped, <code>false</code>
+        /// othwerwise (the instance with given <code>name</code> was not found).</returns>
+        public static bool Stop(string name, bool cancel)
+        {
+            lock (SyncRoot)
+            {
+                NodeKey key = new NodeKey(name);
+
+                Ignite node;
+
+                if (!Nodes.TryGetValue(key, out node))
+                    return false;
+
+                node.Stop(cancel);
+
+                Nodes.Remove(key);
+                
+                GC.Collect();
+
+                return true;
+            }
+        }
+
+        /// <summary>
+        /// Stops <b>all</b> started grids. If <code>cancel</code> flag is set to <code>true</code> then
+        /// all jobs currently executing on local node will be interrupted.
+        /// </summary>
+        /// <param name="cancel">If <code>true</code> then all jobs currently executing will be cancelled
+        /// by calling <code>ComputeJob.cancel</code>method.</param>
+        public static void StopAll(bool cancel)
+        {
+            lock (SyncRoot)
+            {
+                while (Nodes.Count > 0)
+                {
+                    var entry = Nodes.First();
+                    
+                    entry.Value.Stop(cancel);
+
+                    Nodes.Remove(entry.Key);
+                }
+            }
+
+            GC.Collect();
+        }
+        
+        /// <summary>
+        /// Handles the AssemblyResolve event of the CurrentDomain control.
+        /// </summary>
+        /// <param name="sender">The source of the event.</param>
+        /// <param name="args">The <see cref="ResolveEventArgs"/> instance containing the event data.</param>
+        /// <returns>Manually resolved assembly, or null.</returns>
+        private static Assembly CurrentDomain_AssemblyResolve(object sender, ResolveEventArgs args)
+        {
+            return LoadedAssembliesResolver.Instance.GetAssembly(args.Name);
+        }
+
+        /// <summary>
+        /// Grid key.
+        /// </summary>
+        private class NodeKey
+        {
+            /** */
+            private readonly string _name;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="NodeKey"/> class.
+            /// </summary>
+            /// <param name="name">The name.</param>
+            internal NodeKey(string name)
+            {
+                _name = name;
+            }
+
+            /** <inheritdoc /> */
+            public override bool Equals(object obj)
+            {
+                var other = obj as NodeKey;
+
+                return other != null && Equals(_name, other._name);
+            }
+
+            /** <inheritdoc /> */
+            public override int GetHashCode()
+            {
+                return _name == null ? 0 : _name.GetHashCode();
+            }
+        }
+
+        /// <summary>
+        /// Value object to pass data between .Net methods during startup bypassing Java.
+        /// </summary>
+        private unsafe class Startup
+        {
+            /// <summary>
+            /// Constructor.
+            /// </summary>
+            /// <param name="cfg">Configuration.</param>
+            internal Startup(IgniteConfiguration cfg)
+            {
+                Configuration = cfg;
+            }
+
+            /// <summary>
+            /// Configuration.
+            /// </summary>
+            internal IgniteConfiguration Configuration { get; private set; }
+
+            /// <summary>
+            /// Lifecycle beans.
+            /// </summary>
+            internal IList<LifecycleBeanHolder> LifecycleBeans { get; set; }
+
+            /// <summary>
+            /// Node name.
+            /// </summary>
+            internal string Name { get; set; }
+
+            /// <summary>
+            /// Marshaller.
+            /// </summary>
+            internal PortableMarshaller Marshaller { get; set; }
+
+            /// <summary>
+            /// Start error.
+            /// </summary>
+            internal Exception Error { get; set; }
+
+            /// <summary>
+            /// Gets or sets the context.
+            /// </summary>
+            internal void* Context { get; set; }
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheAffinityImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheAffinityImpl.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheAffinityImpl.cs
new file mode 100644
index 0000000..6d577ce
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheAffinityImpl.cs
@@ -0,0 +1,275 @@
+/*
+ * 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.Cache
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+    using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils;
+
+    /// <summary>
+    /// Cache affinity implementation.
+    /// </summary>
+    internal class CacheAffinityImpl : PlatformTarget, ICacheAffinity
+    {
+        /** */
+        private const int OpAffinityKey = 1;
+
+        /** */
+        private const int OpAllPartitions = 2;
+
+        /** */
+        private const int OpBackupPartitions = 3;
+
+        /** */
+        private const int OpIsBackup = 4;
+
+        /** */
+        private const int OpIsPrimary = 5;
+
+        /** */
+        private const int OpIsPrimaryOrBackup = 6;
+
+        /** */
+        private const int OpMapKeyToNode = 7;
+
+        /** */
+        private const int OpMapKeyToPrimaryAndBackups = 8;
+
+        /** */
+        private const int OpMapKeysToNodes = 9;
+
+        /** */
+        private const int OpMapPartitionToNode = 10;
+
+        /** */
+        private const int OpMapPartitionToPrimaryAndBackups = 11;
+
+        /** */
+        private const int OpMapPartitionsToNodes = 12;
+
+        /** */
+        private const int OpPartition = 13;
+
+        /** */
+        private const int OpPrimaryPartitions = 14;
+
+        /** */
+        private readonly bool _keepPortable;
+        
+        /** Grid. */
+        private readonly Ignite _ignite;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheAffinityImpl" /> class.
+        /// </summary>
+        /// <param name="target">Target.</param>
+        /// <param name="marsh">Marshaller.</param>
+        /// <param name="keepPortable">Keep portable flag.</param>
+        /// <param name="ignite">Grid.</param>
+        public CacheAffinityImpl(IUnmanagedTarget target, PortableMarshaller marsh, bool keepPortable, 
+            Ignite ignite) : base(target, marsh)
+        {
+            _keepPortable = keepPortable;
+
+            Debug.Assert(ignite != null);
+            
+            _ignite = ignite;
+        }
+
+        /** <inheritDoc /> */
+        public int Partitions
+        {
+            get { return UU.AffinityPartitions(Target); }
+        }
+
+        /** <inheritDoc /> */
+        public int Partition<TK>(TK key)
+        {
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            return (int)DoOutOp(OpPartition, key);
+        }
+
+        /** <inheritDoc /> */
+        public bool IsPrimary<TK>(IClusterNode n, TK key)
+        {
+            IgniteArgumentCheck.NotNull(n, "n");
+            
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            return DoOutOp(OpIsPrimary, n.Id, key) == True;
+        }
+
+        /** <inheritDoc /> */
+        public bool IsBackup<TK>(IClusterNode n, TK key)
+        {
+            IgniteArgumentCheck.NotNull(n, "n");
+
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            return DoOutOp(OpIsBackup, n.Id, key) == True;
+        }
+
+        /** <inheritDoc /> */
+        public bool IsPrimaryOrBackup<TK>(IClusterNode n, TK key)
+        {
+            IgniteArgumentCheck.NotNull(n, "n");
+
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            return DoOutOp(OpIsPrimaryOrBackup, n.Id, key) == True;
+        }
+
+        /** <inheritDoc /> */
+        public int[] PrimaryPartitions(IClusterNode n)
+        {
+            IgniteArgumentCheck.NotNull(n, "n");
+
+            return DoOutInOp<Guid, int[]>(OpPrimaryPartitions, n.Id);
+        }
+
+        /** <inheritDoc /> */
+        public int[] BackupPartitions(IClusterNode n)
+        {
+            IgniteArgumentCheck.NotNull(n, "n");
+
+            return DoOutInOp<Guid, int[]>(OpBackupPartitions, n.Id);
+        }
+
+        /** <inheritDoc /> */
+        public int[] AllPartitions(IClusterNode n)
+        {
+            IgniteArgumentCheck.NotNull(n, "n");
+
+            return DoOutInOp<Guid, int[]>(OpAllPartitions, n.Id);
+        }
+
+        /** <inheritDoc /> */
+        public TR AffinityKey<TK, TR>(TK key)
+        {
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            return DoOutInOp<TK, TR>(OpAffinityKey, key);
+        }
+
+        /** <inheritDoc /> */
+        public IDictionary<IClusterNode, IList<TK>> MapKeysToNodes<TK>(IList<TK> keys)
+        {
+            IgniteArgumentCheck.NotNull(keys, "keys");
+
+            return DoOutInOp(OpMapKeysToNodes, w => w.WriteObject(keys),
+                reader => ReadDictionary(reader, ReadNode, r => r.ReadObject<IList<TK>>()));
+        }
+
+        /** <inheritDoc /> */
+        public IClusterNode MapKeyToNode<TK>(TK key)
+        {
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            return GetNode(DoOutInOp<TK, Guid?>(OpMapKeyToNode, key));
+        }
+
+        /** <inheritDoc /> */
+        public IList<IClusterNode> MapKeyToPrimaryAndBackups<TK>(TK key)
+        {
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            return DoOutInOp(OpMapKeyToPrimaryAndBackups, w => w.WriteObject(key), r => ReadNodes(r));
+        }
+
+        /** <inheritDoc /> */
+        public IClusterNode MapPartitionToNode(int part)
+        {
+            return GetNode(DoOutInOp<int, Guid?>(OpMapPartitionToNode, part));
+        }
+
+        /** <inheritDoc /> */
+        public IDictionary<int, IClusterNode> MapPartitionsToNodes(IList<int> parts)
+        {
+            IgniteArgumentCheck.NotNull(parts, "parts");
+
+            return DoOutInOp(OpMapPartitionsToNodes,
+                w => w.WriteObject(parts),
+                reader => ReadDictionary(reader, r => r.ReadInt(), ReadNode));
+        }
+
+        /** <inheritDoc /> */
+        public IList<IClusterNode> MapPartitionToPrimaryAndBackups(int part)
+        {
+            return DoOutInOp(OpMapPartitionToPrimaryAndBackups, w => w.WriteObject(part), r => ReadNodes(r));
+        }
+
+        /** <inheritDoc /> */
+        protected override T Unmarshal<T>(IPortableStream stream)
+        {
+            return Marshaller.Unmarshal<T>(stream, _keepPortable);
+        }
+
+
+        /// <summary>
+        /// Gets the node by id.
+        /// </summary>
+        /// <param name="id">The id.</param>
+        /// <returns>Node.</returns>
+        private IClusterNode GetNode(Guid? id)
+        {
+            return _ignite.GetNode(id);
+        }
+
+        /// <summary>
+        /// Reads a node from stream.
+        /// </summary>
+        private IClusterNode ReadNode(PortableReaderImpl r)
+        {
+            return GetNode(r.ReadGuid());
+        }
+
+        /// <summary>
+        /// Reads nodes from stream.
+        /// </summary>
+        private IList<IClusterNode> ReadNodes(IPortableStream reader)
+        {
+            return IgniteUtils.ReadNodes(Marshaller.StartUnmarshal(reader, _keepPortable));
+        }
+
+        /// <summary>
+        /// Reads a dictionary from stream.
+        /// </summary>
+        private Dictionary<TK, TV> ReadDictionary<TK, TV>(IPortableStream reader, Func<PortableReaderImpl, TK> readKey,
+            Func<PortableReaderImpl, TV> readVal)
+        {
+            var r = Marshaller.StartUnmarshal(reader, _keepPortable);
+
+            var cnt = r.ReadInt();
+
+            var dict = new Dictionary<TK, TV>(cnt);
+
+            for (var i = 0; i < cnt; i++)
+                dict[readKey(r)] = readVal(r);
+
+            return dict;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntry.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntry.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntry.cs
new file mode 100644
index 0000000..e28b3e2
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntry.cs
@@ -0,0 +1,126 @@
+/*
+ * 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.Cache
+{
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Cache;
+
+    /// <summary>
+    /// Represents a cache entry.
+    /// </summary>
+    internal struct CacheEntry<TK, TV> : ICacheEntry<TK, TV>
+    {
+        /** Key. */
+        private readonly TK _key;
+
+        /** Value. */
+        private readonly TV _val;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheEntry{K,V}"/> struct.
+        /// </summary>
+        /// <param name="key">The key.</param>
+        /// <param name="val">The value.</param>
+        public CacheEntry(TK key, TV val)
+        {
+            _key = key;
+            _val = val;
+        }
+
+        /// <summary>
+        /// Gets the key.
+        /// </summary>
+        public TK Key
+        {
+            get { return _key; }
+        }
+
+        /// <summary>
+        /// Gets the value.
+        /// </summary>
+        public TV Value
+        {
+            get { return _val; }
+        }
+
+        /// <summary>
+        /// Determines whether the specified <see cref="CacheEntry{K,V}"/>, is equal to this instance.
+        /// </summary>
+        /// <param name="other">The <see cref="CacheEntry{K,V}"/> to compare with this instance.</param>
+        /// <returns>
+        ///   <c>true</c> if the specified <see cref="CacheEntry{K,V}"/> is equal to this instance; 
+        ///   otherwise, <c>false</c>.
+        /// </returns>
+        public bool Equals(CacheEntry<TK, TV> other)
+        {
+            return EqualityComparer<TK>.Default.Equals(_key, other._key) &&
+                EqualityComparer<TV>.Default.Equals(_val, other._val);
+        }
+        
+        /** <inheritDoc /> */
+        public override bool Equals(object obj)
+        {
+            if (ReferenceEquals(null, obj)) 
+                return false;
+
+            return obj is CacheEntry<TK, TV> && Equals((CacheEntry<TK, TV>) obj);
+        }
+        
+        /** <inheritDoc /> */
+        public override int GetHashCode()
+        {
+            unchecked
+            {
+                return (EqualityComparer<TK>.Default.GetHashCode(_key) * 397) ^
+                    EqualityComparer<TV>.Default.GetHashCode(_val);
+            }
+        }
+
+        /** <inheritDoc /> */
+        public override string ToString()
+        {
+            return string.Format("CacheEntry [Key={0}, Value={1}]", _key, _val);
+        }
+
+        /// <summary>
+        /// Implements the operator ==.
+        /// </summary>
+        /// <param name="a">First item.</param>
+        /// <param name="b">Second item.</param>
+        /// <returns>
+        /// The result of the operator.
+        /// </returns>
+        public static bool operator ==(CacheEntry<TK, TV> a, CacheEntry<TK, TV> b)
+        {
+            return a.Equals(b);
+        }
+
+        /// <summary>
+        /// Implements the operator !=.
+        /// </summary>
+        /// <param name="a">First item.</param>
+        /// <param name="b">Second item.</param>
+        /// <returns>
+        /// The result of the operator.
+        /// </returns>
+        public static bool operator !=(CacheEntry<TK, TV> a, CacheEntry<TK, TV> b)
+        {
+            return !(a == b);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntryFilterHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntryFilterHolder.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntryFilterHolder.cs
new file mode 100644
index 0000000..1181645
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntryFilterHolder.cs
@@ -0,0 +1,147 @@
+/*
+ * 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.Cache
+{
+    using System;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Non-generic portable filter wrapper.
+    /// </summary>
+    internal class CacheEntryFilterHolder : IPortableWriteAware
+    {
+        /** Wrapped ICacheEntryFilter */
+        private readonly object _pred;
+
+        /** Invoker function that takes key and value and invokes wrapped ICacheEntryFilter */
+        private readonly Func<object, object, bool> _invoker;
+        
+        /** Keep portable flag. */
+        private readonly bool _keepPortable;
+
+        /** Grid. */
+        private readonly PortableMarshaller _marsh;
+        
+        /** Handle. */
+        private readonly long _handle;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheEntryFilterHolder" /> class.
+        /// </summary>
+        /// <param name="pred">The <see cref="ICacheEntryFilter{TK,TV}" /> to wrap.</param>
+        /// <param name="invoker">The invoker func that takes key and value and invokes wrapped ICacheEntryFilter.</param>
+        /// <param name="marsh">Marshaller.</param>
+        /// <param name="keepPortable">Keep portable flag.</param>
+        public CacheEntryFilterHolder(object pred, Func<object, object, bool> invoker, PortableMarshaller marsh, 
+            bool keepPortable)
+        {
+            Debug.Assert(pred != null);
+            Debug.Assert(invoker != null);
+            Debug.Assert(marsh != null);
+
+            _pred = pred;
+            _invoker = invoker;
+            _marsh = marsh;
+            _keepPortable = keepPortable;
+
+            _handle = marsh.Ignite.HandleRegistry.Allocate(this);
+        }
+
+        /// <summary>
+        /// Gets the handle.
+        /// </summary>
+        public long Handle
+        {
+            get { return _handle; }
+        }
+
+        /// <summary>
+        /// Invokes the cache filter.
+        /// </summary>
+        /// <param name="input">The input stream.</param>
+        /// <returns>Invocation result.</returns>
+        public int Invoke(IPortableStream input)
+        {
+            var rawReader = _marsh.StartUnmarshal(input, _keepPortable).RawReader();
+
+            return _invoker(rawReader.ReadObject<object>(), rawReader.ReadObject<object>()) ? 1 : 0;
+        }
+
+        /** <inheritdoc /> */
+        public void WritePortable(IPortableWriter writer)
+        {
+            var writer0 = (PortableWriterImpl)writer.RawWriter();
+
+            writer0.DetachNext();
+            PortableUtils.WritePortableOrSerializable(writer0, _pred);
+            
+            writer0.WriteBoolean(_keepPortable);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheEntryFilterHolder"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        public CacheEntryFilterHolder(IPortableReader reader)
+        {
+            var reader0 = (PortableReaderImpl)reader.RawReader();
+
+            _pred = PortableUtils.ReadPortableOrSerializable<object>(reader0);
+
+            _keepPortable = reader0.ReadBoolean();
+
+            _marsh = reader0.Marshaller;
+
+            _invoker = GetInvoker(_pred);
+
+            _handle = _marsh.Ignite.HandleRegistry.Allocate(this);
+        }
+
+        /// <summary>
+        /// Gets the invoker func.
+        /// </summary>
+        private static Func<object, object, bool> GetInvoker(object pred)
+        {
+            var func = DelegateTypeDescriptor.GetCacheEntryFilter(pred.GetType());
+
+            return (key, val) => func(pred, key, val);
+        }
+
+        /// <summary>
+        /// Creates an instance of this class from a stream.
+        /// </summary>
+        /// <param name="memPtr">Memory pointer.</param>
+        /// <param name="grid">Grid.</param>
+        /// <returns>Deserialized instance of <see cref="CacheEntryFilterHolder"/></returns>
+        public static CacheEntryFilterHolder CreateInstance(long memPtr, Ignite grid)
+        {
+            var stream = IgniteManager.Memory.Get(memPtr).Stream();
+
+            Debug.Assert(grid != null);
+
+            var marsh = grid.Marshaller;
+
+            return marsh.Unmarshal<CacheEntryFilterHolder>(stream);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntryProcessorHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntryProcessorHolder.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntryProcessorHolder.cs
new file mode 100644
index 0000000..4ec1e1e
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntryProcessorHolder.cs
@@ -0,0 +1,145 @@
+/*
+ * 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.Cache
+{
+    using System;
+    using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Reflection;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Resource;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Portable wrapper for the <see cref="ICacheEntryProcessor{TK,TV,TA,TR}"/> and it's argument.
+    /// Marshals and executes wrapped processor with a non-generic interface.
+    /// </summary>
+    internal class CacheEntryProcessorHolder : IPortableWriteAware
+    {
+        // generic processor
+        private readonly object _proc;
+
+        // argument
+        private readonly object _arg;
+
+        // func to invoke Process method on ICacheEntryProcessor in form of object.
+        private readonly Func<IMutableCacheEntryInternal, object, object> _processFunc;
+
+        // entry creator delegate
+        private readonly Func<object, object, bool, IMutableCacheEntryInternal> _entryCtor;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheEntryProcessorHolder"/> class.
+        /// </summary>
+        /// <param name="proc">The processor to wrap.</param>
+        /// <param name="arg">The argument.</param>
+        /// <param name="processFunc">Delegate to call generic <see cref="ICacheEntryProcessor{K, V, A, R}.Process"/> on local node.</param>
+        /// <param name="keyType">Type of the key.</param>
+        /// <param name="valType">Type of the value.</param>
+        public CacheEntryProcessorHolder(object proc, object arg, 
+            Func<IMutableCacheEntryInternal, object, object> processFunc, Type keyType, Type valType)
+        {
+            Debug.Assert(proc != null);
+            Debug.Assert(processFunc != null);
+
+            _proc = proc;
+            _arg = arg;
+            _processFunc = processFunc;
+
+            _processFunc = GetProcessFunc(_proc);
+
+            _entryCtor = MutableCacheEntry.GetCtor(keyType, valType);
+        }
+
+        /// <summary>
+        /// Processes specified cache entry.
+        /// </summary>
+        /// <param name="key">The cache entry key.</param>
+        /// <param name="value">The cache entry value.</param>
+        /// <param name="exists">Indicates whether cache entry exists.</param>
+        /// <param name="grid"></param>
+        /// <returns>
+        /// Pair of resulting cache entry and result of processing it.
+        /// </returns>
+        [SuppressMessage("Microsoft.Design", "CA1031:DoNotCatchGeneralExceptionTypes", 
+            Justification = "User processor can throw any exception")]
+        public CacheEntryProcessorResultHolder Process(object key, object value, bool exists, Ignite grid)
+        {
+            ResourceProcessor.Inject(_proc, grid);
+
+            var entry = _entryCtor(key, value, exists);
+
+            try
+            {
+                return new CacheEntryProcessorResultHolder(entry, _processFunc(entry, _arg), null);
+            }
+            catch (TargetInvocationException ex)
+            {
+                return new CacheEntryProcessorResultHolder(null, null, ex.InnerException);
+            }
+            catch (Exception ex)
+            {
+                return new CacheEntryProcessorResultHolder(null, null, ex);
+            }
+        }
+
+        /** <inheritDoc /> */
+        public void WritePortable(IPortableWriter writer)
+        {
+            var writer0 = (PortableWriterImpl) writer.RawWriter();
+
+            writer0.DetachNext();
+            PortableUtils.WritePortableOrSerializable(writer0, _proc);
+            PortableUtils.WritePortableOrSerializable(writer0, _arg);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheEntryProcessorHolder"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        public CacheEntryProcessorHolder(IPortableReader reader)
+        {
+            var reader0 = (PortableReaderImpl) reader.RawReader();
+
+            _proc = PortableUtils.ReadPortableOrSerializable<object>(reader0);
+            _arg = PortableUtils.ReadPortableOrSerializable<object>(reader0);
+
+            _processFunc = GetProcessFunc(_proc);
+
+            var kvTypes = DelegateTypeDescriptor.GetCacheEntryProcessorTypes(_proc.GetType());
+
+            _entryCtor = MutableCacheEntry.GetCtor(kvTypes.Item1, kvTypes.Item2);
+        }
+
+        /// <summary>
+        /// Gets a delegate to call generic <see cref="ICacheEntryProcessor{K, V, A, R}.Process"/>.
+        /// </summary>
+        /// <param name="proc">The processor instance.</param>
+        /// <returns>
+        /// Delegate to call generic <see cref="ICacheEntryProcessor{K, V, A, R}.Process"/>.
+        /// </returns>
+        private static Func<IMutableCacheEntryInternal, object, object> GetProcessFunc(object proc)
+        {
+            var func = DelegateTypeDescriptor.GetCacheEntryProcessor(proc.GetType());
+            
+            return (entry, arg) => func(proc, entry, arg);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntryProcessorResult.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntryProcessorResult.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntryProcessorResult.cs
new file mode 100644
index 0000000..9a0af4f
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntryProcessorResult.cs
@@ -0,0 +1,65 @@
+/*
+ * 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.Cache
+{
+    using System;
+    using Apache.Ignite.Core.Cache;
+
+    /// <summary>
+    /// Represents a result of <see cref="ICacheEntryProcessor{TK,TV,TA,TR}"/> invocation.
+    /// </summary>
+    /// <typeparam name="T">Result type.</typeparam>
+    internal class CacheEntryProcessorResult<T> : ICacheEntryProcessorResult<T>
+    {
+        // Result
+        private readonly T _res;
+
+        // Error
+        private readonly Exception _err;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheEntryProcessorResult{T}"/> class.
+        /// </summary>
+        /// <param name="result">The result.</param>
+        public CacheEntryProcessorResult(T result)
+        {
+            _res = result;
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheEntryProcessorResult{T}"/> class.
+        /// </summary>
+        /// <param name="error">The error.</param>
+        public CacheEntryProcessorResult(Exception error)
+        {
+            _err = error;
+        }
+
+        /** <inheritdoc /> */
+        public T Result
+        {
+            get
+            {
+                if (_err != null)
+                    throw _err as CacheEntryProcessorException ?? new CacheEntryProcessorException(_err);
+
+                return _res;
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntryProcessorResultHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntryProcessorResultHolder.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntryProcessorResultHolder.cs
new file mode 100644
index 0000000..04cd557
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntryProcessorResultHolder.cs
@@ -0,0 +1,127 @@
+/*
+ * 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.Cache
+{
+    using System;
+    using System.Diagnostics.CodeAnalysis;
+    using System.IO;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+
+    /// <summary>
+    /// Manages cache entry processing result in non-generic form.
+    /// </summary>
+    internal class CacheEntryProcessorResultHolder
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheEntryProcessorResultHolder"/> class.
+        /// </summary>
+        /// <param name="entry">Entry.</param>
+        /// <param name="processResult">Process result.</param>
+        /// <param name="error">Error.</param>
+        public CacheEntryProcessorResultHolder(IMutableCacheEntryInternal entry, object processResult, Exception error)
+        {
+            Entry = entry;
+            ProcessResult = processResult;
+            Error = error;
+        }
+
+        /// <summary>
+        /// Gets the entry.
+        /// </summary>
+        public IMutableCacheEntryInternal Entry { get; private set; }
+
+        /// <summary>
+        /// Gets the process result.
+        /// </summary>
+        public object ProcessResult { get; private set; }
+
+        /// <summary>
+        /// Gets the error.
+        /// </summary>
+        public Exception Error { get; private set; }
+
+        /// <summary>
+        /// Writes this instance to the stream.
+        /// </summary>
+        /// <param name="stream">Stream.</param>
+        /// <param name="marsh">Marshaller.</param>
+        public void Write(IPortableStream stream, PortableMarshaller marsh)
+        {
+            var writer = marsh.StartMarshal(stream);
+
+            try
+            {
+                Marshal(writer);
+            }
+            finally
+            {
+                marsh.FinishMarshal(writer);
+            }
+        }
+
+        /// <summary>
+        /// Marshal this instance.
+        /// </summary>
+        /// <param name="writer">Writer.</param>
+        [SuppressMessage("Microsoft.Design", "CA1031:DoNotCatchGeneralExceptionTypes",
+            Justification = "Any kind of exception can be thrown during user type marshalling.")]
+        private void Marshal(PortableWriterImpl writer)
+        {
+            var pos = writer.Stream.Position;
+
+            try
+            {
+                if (Error == null)
+                {
+                    writer.WriteByte((byte) Entry.State);
+
+                    if (Entry.State == MutableCacheEntryState.ValueSet)
+                        writer.Write(Entry.Value);
+
+                    writer.Write(ProcessResult);
+                }
+                else
+                {
+                    writer.WriteByte((byte) MutableCacheEntryState.ErrPortable);
+                    writer.Write(new PortableResultWrapper(Error));
+                }
+            }
+            catch (Exception marshErr)
+            {
+                writer.Stream.Seek(pos, SeekOrigin.Begin);
+
+                writer.WriteByte((byte) MutableCacheEntryState.ErrString);
+
+                if (Error == null)
+                {
+                    writer.WriteString(string.Format(
+                    "CacheEntryProcessor completed with error, but result serialization failed [errType={0}, " +
+                    "err={1}, serializationErrMsg={2}]", marshErr.GetType().Name, marshErr, marshErr.Message));
+                }
+                else
+                {
+                    writer.WriteString(string.Format(
+                    "CacheEntryProcessor completed with error, and error serialization failed [errType={0}, " +
+                    "err={1}, serializationErrMsg={2}]", marshErr.GetType().Name, marshErr, marshErr.Message));
+                }
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEnumerable.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEnumerable.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEnumerable.cs
new file mode 100644
index 0000000..2dd03c9
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEnumerable.cs
@@ -0,0 +1,82 @@
+/*
+ * 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.Cache
+{
+    using System.Collections;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Cache;
+
+    /// <summary>
+    /// Cache enumerable.
+    /// </summary>
+    internal class CacheEnumerable<TK, TV> : IEnumerable<ICacheEntry<TK, TV>>
+    {
+        /** Target cache. */
+        private readonly CacheImpl<TK, TV> _cache;
+
+        /** Local flag. */
+        private readonly bool _loc;
+
+        /** Peek modes. */
+        private readonly int _peekModes;
+
+        /// <summary>
+        /// Constructor for distributed iterator.
+        /// </summary>
+        /// <param name="cache">Target cache.</param>
+        public CacheEnumerable(CacheImpl<TK, TV> cache) : this(cache, false, 0)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Constructor for local iterator.
+        /// </summary>
+        /// <param name="cache">Target cache.</param>
+        /// <param name="peekModes">Peek modes.</param>
+        public CacheEnumerable(CacheImpl<TK, TV> cache, int peekModes) : this(cache, true, peekModes)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="cache">Target cache.</param>
+        /// <param name="loc">Local flag.</param>
+        /// <param name="peekModes">Peek modes.</param>
+        private CacheEnumerable(CacheImpl<TK, TV> cache, bool loc, int peekModes)
+        {
+            _cache = cache;
+            _loc = loc;
+            _peekModes = peekModes;
+        }
+
+        /** <inheritdoc /> */
+        public IEnumerator<ICacheEntry<TK, TV>> GetEnumerator()
+        {
+            return new CacheEnumeratorProxy<TK, TV>(_cache, _loc, _peekModes);
+        }
+
+        /** <inheritdoc /> */
+        IEnumerator IEnumerable.GetEnumerator()
+        {
+            return GetEnumerator();
+        }
+    }
+}


[08/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache-portables.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache-portables.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache-portables.xml
new file mode 100644
index 0000000..1d90c99
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache-portables.xml
@@ -0,0 +1,226 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+
+        <property name="marshaller">
+            <bean class="org.gridgain.grid.marshaller.optimized.GridOptimizedMarshaller">
+                <property name="requireSerializable" value="false"/>
+            </bean>
+        </property>
+
+        <property name="includeEventTypes">
+            <util:constant static-field="org.apache.ignite.events.IgniteEventType.EVTS_CACHE"/>
+        </property>
+
+        <property name="portableConfiguration">
+            <bean class="org.gridgain.grid.configuration.PortableConfiguration">
+                <property name="typeConfigurations">
+                    <list>
+                        <bean class="org.apache.ignite.portable.PortableTypeConfiguration">
+                            <property name="className" value="org.gridgain.client.GridClientTestPortable"/>
+                        </bean>
+                        <bean class="org.apache.ignite.portable.PortableTypeConfiguration">
+                            <property name="className" value="org.gridgain.client.model.PortablePerson"/>
+                        </bean>
+                        <bean class="org.apache.ignite.portable.PortableTypeConfiguration">
+                            <property name="className" value="org.gridgain.client.model.ImplicitPortablePerson"/>
+                        </bean>
+                        <bean class="org.apache.ignite.portable.PortableTypeConfiguration">
+                            <property name="className" value="NoDefPortablePerson"/>
+                        </bean>
+                    </list>
+                </property>
+            </bean>
+        </property>
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean parent="cache-template">
+                    <property name="name" value="local"/>
+                    <property name="cacheMode" value="LOCAL"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                </bean>
+
+                <bean parent="cache-template">
+                    <property name="name" value="local_atomic"/>
+                    <property name="cacheMode" value="LOCAL"/>
+                    <property name="atomicityMode" value="ATOMIC"/>
+                </bean>
+
+                <bean parent="cache-template">
+                    <property name="name" value="partitioned"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                </bean>
+
+                <bean parent="cache-template">
+                    <property name="name" value="partitioned_atomic"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="atomicWriteOrderMode" value="PRIMARY"/>
+                </bean>
+
+                <bean parent="cache-template">
+                    <property name="name" value="partitioned_near"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="nearConfiguration">
+                        <bean class="org.apache.ignite.configuration.NearCacheConfiguration" />
+                    </property>
+                </bean>
+
+                <bean parent="cache-template">
+                    <property name="name" value="partitioned_atomic_near"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="atomicWriteOrderMode" value="PRIMARY"/>
+                    <property name="nearConfiguration">
+                        <bean class="org.apache.ignite.configuration.NearCacheConfiguration" />
+                    </property>
+                </bean>
+
+                <bean parent="cache-template">
+                    <property name="name" value="replicated"/>
+                    <property name="cacheMode" value="REPLICATED"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                </bean>
+
+                <bean parent="cache-template">
+                    <property name="name" value="replicated_atomic"/>
+                    <property name="cacheMode" value="REPLICATED"/>
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="atomicWriteOrderMode" value="PRIMARY"/>
+                </bean>
+            </list>
+        </property>
+
+        <property name="swapSpaceSpi">
+            <bean class="org.gridgain.grid.spi.swapspace.inmemory.GridTestSwapSpaceSpi"/>
+        </property>
+
+        <property name="indexingSpi">
+            <list>
+                <bean class="org.gridgain.grid.spi.indexing.h2.GridH2IndexingSpi">
+                </bean>
+            </list>
+        </property>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47501</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+
+        <property name="platformConfiguration">
+            <bean class="org.apache.ignite.platform.dotnet.PlatformDotNetConfiguration">
+                <property name="portableConfiguration">
+                    <bean class="org.apache.ignite.platform.dotnet.PlatformDotNetPortableConfiguration">
+                        <property name="types">
+                            <util:list value-type="java.lang.String">
+
+                            </util:list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+
+    <bean id="cache-template" abstract="true" class="org.apache.ignite.cache.CacheConfiguration">
+        <property name="rebalanceMode" value="SYNC"/>
+        <property name="writeSynchronizationMode" value="FULL_SYNC"/>
+        <property name="swapEnabled" value="true"/>
+        <property name="portableEnabled" value="true"/>
+        <property name="txSerializableEnabled" value="true"/>
+        <property name="backups" value="1"/>
+
+        <property name="store">
+            <bean class="org.gridgain.grid.internal.processors.cache.GridCacheGenericTestStore"/>
+        </property>
+
+        <property name="typeMetadata">
+            <list>
+                <bean class="org.apache.ignite.cache.CacheTypeMetadata">
+                    <property name="valueType" value="PortablePerson"/>
+                    <property name="ascendingFields">
+                        <map>
+                            <entry key="age" value="java.lang.Integer"/>
+                        </map>
+                    </property>
+                    <property name="queryFields">
+                        <map>
+                            <entry key="name" value="java.lang.String"/>
+                        </map>
+                    </property>
+                    <property name="textFields">
+                        <list>
+                            <value>address</value>
+                        </list>
+                    </property>
+                </bean>
+                <bean class="org.apache.ignite.cache.CacheTypeMetadata">
+                    <property name="valueType" value="ImplicitPortablePerson"/>
+                    <property name="ascendingFields">
+                        <map>
+                            <entry key="age" value="java.lang.Integer"/>
+                        </map>
+                    </property>
+                    <property name="queryFields">
+                        <map>
+                            <entry key="name" value="java.lang.String"/>
+                        </map>
+                    </property>
+                </bean>
+                <bean class="org.apache.ignite.cache.CacheTypeMetadata">
+                    <property name="valueType" value="NoDefPortablePerson"/>
+                    <property name="ascendingFields">
+                        <map>
+                            <entry key="age" value="java.lang.Integer"/>
+                        </map>
+                    </property>
+                    <property name="queryFields">
+                        <map>
+                            <entry key="name" value="java.lang.String"/>
+                        </map>
+                    </property>
+                </bean>
+            </list>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache-store.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache-store.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache-store.xml
new file mode 100644
index 0000000..9af4611
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache-store.xml
@@ -0,0 +1,125 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+
+        <property name="includeEventTypes">
+            <util:constant static-field="org.apache.ignite.events.EventType.EVTS_CACHE"/>
+        </property>
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="portable_store"/>
+                    <property name="cacheMode" value="LOCAL"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="writeThrough" value="true"/>
+                    <property name="readThrough" value="true"/>
+
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.platform.dotnet.PlatformDotNetCacheStoreFactory">
+                            <property name="assemblyName" value="Apache.Ignite.Core.Tests"/>
+                            <property name="className" value="Apache.Ignite.Core.Tests.Cache.Store.CacheTestStore"/>
+                        </bean>
+                    </property>
+                </bean>
+
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="object_store"/>
+                    <property name="cacheMode" value="LOCAL"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="writeThrough" value="true"/>
+                    <property name="readThrough" value="true"/>
+                    <property name="keepPortableInStore" value="false"/>
+
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.platform.dotnet.PlatformDotNetCacheStoreFactory">
+                            <property name="assemblyName" value="Apache.Ignite.Core.Tests"/>
+                            <property name="className" value="Apache.Ignite.Core.Tests.Cache.Store.CacheTestStore"/>
+                        </bean>
+                    </property>
+                </bean>
+
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="template_store*"/>
+                    <property name="cacheMode" value="LOCAL"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="writeThrough" value="true"/>
+                    <property name="readThrough" value="true"/>
+                    <property name="keepPortableInStore" value="false"/>
+
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.platform.dotnet.PlatformDotNetCacheStoreFactory">
+                            <property name="assemblyName" value="Apache.Ignite.Core.Tests"/>
+                            <property name="className" value="Apache.Ignite.Core.Tests.Cache.Store.CacheTestStore"/>
+                        </bean>
+                    </property>
+                </bean>
+
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="custom_store"/>
+                    <property name="cacheMode" value="LOCAL"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="writeThrough" value="true"/>
+                    <property name="readThrough" value="true"/>
+                    
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.platform.dotnet.PlatformDotNetCacheStoreFactory">
+                            <property name="assemblyName" value="Apache.Ignite.Core.Tests"/>
+                            <property name="className" value="Apache.Ignite.Core.Tests.Cache.Store.CacheTestStore"/>
+                            <property name="properties">
+                                <map>
+                                    <entry key="IntProperty">
+                                        <value type="java.lang.Integer">42</value>
+                                    </entry>
+                                    <entry key="StringProperty" value="String value"/>
+                                </map>
+                            </property>
+                        </bean>
+                    </property>
+                </bean>
+            </list>
+        </property>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47501</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache.xml
new file mode 100644
index 0000000..5d65ae8
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache.xml
@@ -0,0 +1,224 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+
+        <property name="marshaller">
+            <bean class="org.apache.ignite.marshaller.portable.PortableMarshaller">
+                <property name="typeConfigurations">
+                    <list>
+                        <bean class="org.apache.ignite.portable.PortableTypeConfiguration">
+                            <property name="className" value="org.gridgain.client.ClientTestPortable"/>
+                        </bean>
+                        <bean class="org.apache.ignite.portable.PortableTypeConfiguration">
+                            <property name="className" value="org.gridgain.internal.client.model.PortablePerson"/>
+                        </bean>
+                        <bean class="org.apache.ignite.portable.PortableTypeConfiguration">
+                            <property name="className" value="org.gridgain.internal.client.model.ImplicitPortablePerson"/>
+                        </bean>
+                        <bean class="org.apache.ignite.portable.PortableTypeConfiguration">
+                            <property name="className" value="NoDefPortablePerson"/>
+                        </bean>
+                    </list>
+                </property>
+            </bean>
+        </property>
+
+        <property name="includeEventTypes">
+            <util:constant static-field="org.apache.ignite.events.EventType.EVTS_CACHE"/>
+        </property>
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean parent="cache-template">
+                    <property name="name" value="local"/>
+                    <property name="cacheMode" value="LOCAL"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="startSize" value="10"/>
+                </bean>
+
+                <bean parent="cache-template">
+                    <property name="name" value="local_atomic"/>
+                    <property name="cacheMode" value="LOCAL"/>
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="startSize" value="10"/>
+                </bean>
+
+                <bean parent="cache-template">
+                    <property name="name" value="partitioned"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="startSize" value="10"/>
+                </bean>
+
+                <bean parent="cache-template">
+                    <property name="name" value="partitioned_atomic"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="atomicWriteOrderMode" value="PRIMARY"/>
+                    <property name="startSize" value="10"/>
+                </bean>
+
+                <bean parent="cache-template">
+                    <property name="name" value="partitioned_near"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="nearConfiguration">
+                        <bean class="org.apache.ignite.configuration.NearCacheConfiguration" />
+                    </property>
+                    <property name="startSize" value="10"/>
+                </bean>
+
+                <bean parent="cache-template">
+                    <property name="name" value="partitioned_atomic_near"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="atomicWriteOrderMode" value="PRIMARY"/>
+                    <property name="nearConfiguration">
+                        <bean class="org.apache.ignite.configuration.NearCacheConfiguration" />
+                    </property>
+                    <property name="startSize" value="10"/>
+                </bean>
+
+                <bean parent="cache-template">
+                    <property name="name" value="replicated"/>
+                    <property name="cacheMode" value="REPLICATED"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="startSize" value="10"/>
+                </bean>
+
+                <bean parent="cache-template">
+                    <property name="name" value="replicated_atomic"/>
+                    <property name="cacheMode" value="REPLICATED"/>
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="atomicWriteOrderMode" value="PRIMARY"/>
+                    <property name="startSize" value="10"/>
+                </bean>
+                
+                <bean parent="cache-template">
+                    <property name="name" value="template*"/>
+                    <property name="startSize" value="10"/>
+                </bean>
+            </list>
+        </property>
+
+        <property name="swapSpaceSpi">
+            <bean class="org.apache.ignite.spi.swapspace.inmemory.GridTestSwapSpaceSpi"/>
+        </property>
+
+        <!-- TODO: -->
+        <!--<property name="indexingSpi">
+            <list>
+                <bean class="org.gridgain.grid.spi.indexing.h2.GridH2IndexingSpi">
+                    <property name="defaultIndexFixedTyping" value="false"/>
+                </bean>
+            </list>
+        </property>-->
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47501</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+
+        <property name="transactionConfiguration">
+            <bean class="org.apache.ignite.configuration.TransactionConfiguration">
+                <property name="txSerializableEnabled" value="true"/>
+            </bean>
+        </property>
+    </bean>
+
+    <bean id="cache-template" abstract="true" class="org.apache.ignite.configuration.CacheConfiguration">
+        <property name="rebalanceMode" value="SYNC"/>
+        <property name="writeSynchronizationMode" value="FULL_SYNC"/>
+        <property name="swapEnabled" value="true"/>
+        <property name="backups" value="1"/>
+        <property name="eagerTtl" value="true"/>
+
+        <!--
+        <property name="typeMetadata">
+            <list>
+                <bean class="org.apache.ignite.cache.CacheTypeMetadata">
+                    <property name="valueType" value="GridPortablePerson"/>
+                    <property name="ascendingFields">
+                        <map>
+                            <entry key="age" value="java.lang.Integer"/>
+                        </map>
+                    </property>
+                    <property name="queryFields">
+                        <map>
+                            <entry key="name" value="java.lang.String"/>
+                        </map>
+                    </property>
+                    <property name="textFields">
+                        <list>
+                            <value>address</value>
+                        </list>
+                    </property>
+                </bean>
+                <bean class="org.apache.ignite.cache.CacheTypeMetadata">
+                    <property name="valueType" value="GridImplicitPortablePerson"/>
+                    <property name="ascendingFields">
+                        <map>
+                            <entry key="age" value="java.lang.Integer"/>
+                        </map>
+                    </property>
+                    <property name="queryFields">
+                        <map>
+                            <entry key="name" value="java.lang.String"/>
+                        </map>
+                    </property>
+                </bean>
+                <bean class="org.apache.ignite.cache.CacheTypeMetadata">
+                    <property name="valueType" value="GridNoDefPortablePerson"/>
+                    <property name="ascendingFields">
+                        <map>
+                            <entry key="age" value="java.lang.Integer"/>
+                        </map>
+                    </property>
+                    <property name="queryFields">
+                        <map>
+                            <entry key="name" value="java.lang.String"/>
+                        </map>
+                    </property>
+                </bean>
+            </list>
+        </property>
+        -->
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/portable.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/portable.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/portable.xml
new file mode 100644
index 0000000..f013749
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/portable.xml
@@ -0,0 +1,56 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+
+        <property name="gridName" value="grid"/>
+
+        <property name="metricsUpdateFrequency" value="1000"/>
+        <property name="metricsLogFrequency" value="0"/>
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache"/>
+                </bean>
+            </list>
+        </property>
+      
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47502</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/start-test-grid1.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/start-test-grid1.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/start-test-grid1.xml
new file mode 100644
index 0000000..8559173
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/start-test-grid1.xml
@@ -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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+
+        <property name="gridName" value="grid1"/>
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache1"/>
+                    <property name="rebalanceMode" value="SYNC"/>
+                </bean>
+            </list>
+        </property>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47502</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/start-test-grid2.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/start-test-grid2.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/start-test-grid2.xml
new file mode 100644
index 0000000..cb628fe
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/start-test-grid2.xml
@@ -0,0 +1,45 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+
+        <property name="gridName" value="grid2"/>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47502</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/start-test-grid3.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/start-test-grid3.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/start-test-grid3.xml
new file mode 100644
index 0000000..753fad1
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/start-test-grid3.xml
@@ -0,0 +1,43 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47502</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTest.cs
new file mode 100644
index 0000000..3c1619b
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTest.cs
@@ -0,0 +1,592 @@
+/*
+ * 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.Dataload
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using System.Linq;
+    using System.Threading;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Datastream;
+    using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.Portable;
+    using Apache.Ignite.Core.Tests.Cache;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Data streamer tests.
+    /// </summary>
+    public class DataStreamerTest
+    {
+        /** Node name. */
+        protected const string GridName = "grid";
+
+        /** Cache name. */
+        protected const string CacheName = "partitioned";
+
+        /** Node. */
+        private IIgnite _grid;
+
+        /** Cache. */
+        private ICache<int, int?> _cache;
+
+        /// <summary>
+        /// Initialization routine.
+        /// </summary>
+        [TestFixtureSetUp]
+        public virtual void InitClient()
+        {
+            _grid = Ignition.Start(GetIgniteConfiguration(GridName));
+
+            Ignition.Start(GetIgniteConfiguration(GridName + "_1"));
+
+            _cache = _grid.Cache<int, int?>(CacheName);
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        [TestFixtureTearDown]
+        public virtual void StopGrids()
+        {
+            Ignition.StopAll(true);
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        [SetUp]
+        public virtual void BeforeTest()
+        {
+            Console.WriteLine("Test started: " + TestContext.CurrentContext.Test.Name);
+
+            for (int i = 0; i < 100; i++)
+                _cache.Remove(i);
+        }
+
+        [TearDown]
+        public void AfterTest()
+        {
+            TestUtils.AssertHandleRegistryIsEmpty(_grid, 1000);
+        }
+
+        /// <summary>
+        /// Test data streamer property configuration. Ensures that at least no exceptions are thrown.
+        /// </summary>
+        [Test]
+        public void TestPropertyPropagation()
+        {
+            using (IDataStreamer<int, int> ldr = _grid.DataStreamer<int, int>(CacheName))
+            {
+                ldr.AllowOverwrite = true;
+                Assert.IsTrue(ldr.AllowOverwrite);
+                ldr.AllowOverwrite = false;
+                Assert.IsFalse(ldr.AllowOverwrite);
+
+                ldr.SkipStore = true;
+                Assert.IsTrue(ldr.SkipStore);
+                ldr.SkipStore = false;
+                Assert.IsFalse(ldr.SkipStore);
+
+                ldr.PerNodeBufferSize = 1;
+                Assert.AreEqual(1, ldr.PerNodeBufferSize);
+                ldr.PerNodeBufferSize = 2;
+                Assert.AreEqual(2, ldr.PerNodeBufferSize);
+
+                ldr.PerNodeParallelOperations = 1;
+                Assert.AreEqual(1, ldr.PerNodeParallelOperations);
+                ldr.PerNodeParallelOperations = 2;
+                Assert.AreEqual(2, ldr.PerNodeParallelOperations);
+            }
+        }
+
+        /// <summary>
+        /// Test data add/remove.
+        /// </summary>
+        [Test]        
+        public void TestAddRemove()
+        {
+            using (IDataStreamer<int, int> ldr = _grid.DataStreamer<int, int>(CacheName))
+            {
+                ldr.AllowOverwrite = true;
+
+                // Additions.
+                ldr.AddData(1, 1);
+                ldr.Flush();                
+                Assert.AreEqual(1, _cache.Get(1));
+
+                ldr.AddData(new KeyValuePair<int, int>(2, 2));
+                ldr.Flush();
+                Assert.AreEqual(2, _cache.Get(2));
+
+                ldr.AddData(new List<KeyValuePair<int, int>> { new KeyValuePair<int, int>(3, 3), new KeyValuePair<int, int>(4, 4) });
+                ldr.Flush();
+                Assert.AreEqual(3, _cache.Get(3));
+                Assert.AreEqual(4, _cache.Get(4));
+
+                // Removal.
+                ldr.RemoveData(1);
+                ldr.Flush();
+                Assert.IsNull(_cache.Get(1));
+
+                // Mixed.
+                ldr.AddData(5, 5);                
+                ldr.RemoveData(2);
+                ldr.AddData(new KeyValuePair<int, int>(7, 7));
+                ldr.AddData(6, 6);
+                ldr.RemoveData(4);
+                ldr.AddData(new List<KeyValuePair<int, int>> { new KeyValuePair<int, int>(9, 9), new KeyValuePair<int, int>(10, 10) });
+                ldr.AddData(new KeyValuePair<int, int>(8, 8));
+                ldr.RemoveData(3);
+                ldr.AddData(new List<KeyValuePair<int, int>> { new KeyValuePair<int, int>(11, 11), new KeyValuePair<int, int>(12, 12) });
+
+                ldr.Flush();
+
+                for (int i = 2; i < 5; i++)
+                    Assert.IsNull(_cache.Get(i));
+
+                for (int i = 5; i < 13; i++)
+                    Assert.AreEqual(i, _cache.Get(i));
+            }
+        }
+
+        /// <summary>
+        /// Test "tryFlush".
+        /// </summary>
+        [Test]
+        public void TestTryFlush()
+        {
+            using (IDataStreamer<int, int> ldr = _grid.DataStreamer<int, int>(CacheName))
+            {
+                var fut = ldr.AddData(1, 1);
+
+                ldr.TryFlush();
+
+                fut.Get();
+
+                Assert.AreEqual(1, _cache.Get(1));
+            }
+        }
+
+        /// <summary>
+        /// Test buffer size adjustments.
+        /// </summary>
+        [Test]
+        public void TestBufferSize()
+        {
+            using (IDataStreamer<int, int> ldr = _grid.DataStreamer<int, int>(CacheName))
+            {
+                var fut = ldr.AddData(1, 1);
+
+                Thread.Sleep(100);
+
+                Assert.IsFalse(fut.IsDone);
+
+                ldr.PerNodeBufferSize = 2;
+
+                ldr.AddData(2, 2);
+                ldr.AddData(3, 3);
+                ldr.AddData(4, 4).Get();
+                fut.Get();
+
+                Assert.AreEqual(1, _cache.Get(1));
+                Assert.AreEqual(2, _cache.Get(2));
+                Assert.AreEqual(3, _cache.Get(3));
+                Assert.AreEqual(4, _cache.Get(4));
+
+                ldr.AddData(new List<KeyValuePair<int, int>>
+                {
+                    new KeyValuePair<int, int>(5, 5), 
+                    new KeyValuePair<int, int>(6, 6),
+                    new KeyValuePair<int, int>(7, 7), 
+                    new KeyValuePair<int, int>(8, 8)
+                }).Get();
+
+                Assert.AreEqual(5, _cache.Get(5));
+                Assert.AreEqual(6, _cache.Get(6));
+                Assert.AreEqual(7, _cache.Get(7));
+                Assert.AreEqual(8, _cache.Get(8));
+            }
+        }
+
+        /// <summary>
+        /// Test close.
+        /// </summary>
+        [Test]
+        public void TestClose()
+        {
+            using (IDataStreamer<int, int> ldr = _grid.DataStreamer<int, int>(CacheName))
+            {
+                var fut = ldr.AddData(1, 1);
+
+                ldr.Close(false);
+
+                fut.Get();
+
+                Assert.AreEqual(1, _cache.Get(1));
+            }
+        }
+
+        /// <summary>
+        /// Test close with cancellation.
+        /// </summary>
+        [Test]
+        public void TestCancel()
+        {
+            using (IDataStreamer<int, int> ldr = _grid.DataStreamer<int, int>(CacheName))
+            {
+                var fut = ldr.AddData(1, 1);
+
+                ldr.Close(true);
+
+                fut.Get();
+
+                Assert.IsNull(_cache.Get(1));
+            }
+        }
+
+        /// <summary>
+        /// Tests that streamer gets collected when there are no references to it.
+        /// </summary>
+        [Test]
+        public void TestFinalizer()
+        {
+            var streamer = _grid.DataStreamer<int, int>(CacheName);
+            var streamerRef = new WeakReference(streamer);
+
+            Assert.IsNotNull(streamerRef.Target);
+
+            // ReSharper disable once RedundantAssignment
+            streamer = null;
+
+            GC.Collect();
+            GC.WaitForPendingFinalizers();
+
+            Assert.IsNull(streamerRef.Target);
+        }
+
+        /// <summary>
+        /// Test auto-flush feature.
+        /// </summary>
+        [Test]
+        public void TestAutoFlush()
+        {
+            using (IDataStreamer<int, int> ldr = _grid.DataStreamer<int, int>(CacheName))
+            {
+                // Test auto flush turning on.
+                var fut = ldr.AddData(1, 1);
+                Thread.Sleep(100);
+                Assert.IsFalse(fut.IsDone);
+                ldr.AutoFlushFrequency = 1000;                
+                fut.Get();
+
+                // Test forced flush after frequency change.
+                fut = ldr.AddData(2, 2);
+                ldr.AutoFlushFrequency = long.MaxValue;                
+                fut.Get();
+
+                // Test another forced flush after frequency change.
+                fut = ldr.AddData(3, 3);
+                ldr.AutoFlushFrequency = 1000;
+                fut.Get();
+
+                // Test flush before stop.
+                fut = ldr.AddData(4, 4);
+                ldr.AutoFlushFrequency = 0;
+                fut.Get();
+
+                // Test flush after second turn on.
+                fut = ldr.AddData(5, 5);
+                ldr.AutoFlushFrequency = 1000;
+                fut.Get();
+
+                Assert.AreEqual(1, _cache.Get(1));
+                Assert.AreEqual(2, _cache.Get(2));
+                Assert.AreEqual(3, _cache.Get(3));
+                Assert.AreEqual(4, _cache.Get(4));
+                Assert.AreEqual(5, _cache.Get(5));
+            }
+        }
+
+        /// <summary>
+        /// Test multithreaded behavior. 
+        /// </summary>
+        [Test]
+        [Category(TestUtils.CategoryIntensive)]
+        public void TestMultithreaded()
+        {
+            int entriesPerThread = 100000;
+            int threadCnt = 8;
+
+            for (int i = 0; i < 5; i++)
+            {
+                _cache.Clear();
+
+                Assert.AreEqual(0, _cache.Size());
+
+                Stopwatch watch = new Stopwatch();
+
+                watch.Start();
+
+                using (IDataStreamer<int, int> ldr = _grid.DataStreamer<int, int>(CacheName))
+                {
+                    ldr.PerNodeBufferSize = 1024;
+
+                    int ctr = 0;
+
+                    TestUtils.RunMultiThreaded(() =>
+                    {
+                        int threadIdx = Interlocked.Increment(ref ctr);
+
+                        int startIdx = (threadIdx - 1) * entriesPerThread;
+                        int endIdx = startIdx + entriesPerThread;
+
+                        for (int j = startIdx; j < endIdx; j++)
+                        {
+                            // ReSharper disable once AccessToDisposedClosure
+                            ldr.AddData(j, j);
+
+                            if (j % 100000 == 0)
+                                Console.WriteLine("Put [thread=" + threadIdx + ", cnt=" + j  + ']');
+                        }
+                    }, threadCnt);
+                }
+
+                Console.WriteLine("Iteration " + i + ": " + watch.ElapsedMilliseconds);
+
+                watch.Reset();
+
+                for (int j = 0; j < threadCnt * entriesPerThread; j++)
+                    Assert.AreEqual(j, j);
+            }
+        }
+
+        /// <summary>
+        /// Tests custom receiver.
+        /// </summary>
+        [Test]
+        public void TestStreamReceiver()
+        {
+            TestStreamReceiver(new StreamReceiverPortable());
+            TestStreamReceiver(new StreamReceiverSerializable());
+        }
+
+        /// <summary>
+        /// Tests StreamVisitor.
+        /// </summary>
+        [Test]
+        public void TestStreamVisitor()
+        {
+            TestStreamReceiver(new StreamVisitor<int, int>((c, e) => c.Put(e.Key, e.Value + 1)));
+        }
+
+        /// <summary>
+        /// Tests StreamTransformer.
+        /// </summary>
+        [Test]
+        public void TestStreamTransformer()
+        {
+            TestStreamReceiver(new StreamTransformer<int, int, int, int>(new EntryProcessorSerializable()));
+            TestStreamReceiver(new StreamTransformer<int, int, int, int>(new EntryProcessorPortable()));
+        }
+
+        /// <summary>
+        /// Tests specified receiver.
+        /// </summary>
+        private void TestStreamReceiver(IStreamReceiver<int, int> receiver)
+        {
+            using (var ldr = _grid.DataStreamer<int, int>(CacheName))
+            {
+                ldr.AllowOverwrite = true;
+
+                ldr.Receiver = new StreamReceiverPortable();
+
+                ldr.Receiver = receiver;  // check double assignment
+
+                Assert.AreEqual(ldr.Receiver, receiver);
+
+                for (var i = 0; i < 100; i++)
+                    ldr.AddData(i, i);
+
+                ldr.Flush();
+
+                for (var i = 0; i < 100; i++)
+                    Assert.AreEqual(i + 1, _cache.Get(i));
+            }
+        }
+
+        /// <summary>
+        /// Tests the stream receiver in keepPortable mode.
+        /// </summary>
+        [Test]
+        public void TestStreamReceiverKeepPortable()
+        {
+            // ReSharper disable once LocalVariableHidesMember
+            var cache = _grid.Cache<int, PortableEntry>(CacheName);
+
+            using (var ldr0 = _grid.DataStreamer<int, int>(CacheName))
+            using (var ldr = ldr0.WithKeepPortable<int, IPortableObject>())
+            {
+                ldr.Receiver = new StreamReceiverKeepPortable();
+
+                ldr.AllowOverwrite = true;
+
+                for (var i = 0; i < 100; i++)
+                    ldr.AddData(i, _grid.Portables().ToPortable<IPortableObject>(new PortableEntry {Val = i}));
+
+                ldr.Flush();
+
+                for (var i = 0; i < 100; i++)
+                    Assert.AreEqual(i + 1, cache.Get(i).Val);
+            }
+        }
+
+        /// <summary>
+        /// Gets the Ignite configuration.
+        /// </summary>
+        /// <param name="gridName">Grid name.</param>
+        private static IgniteConfigurationEx GetIgniteConfiguration(string gridName)
+        {
+            return new IgniteConfigurationEx
+            {
+                GridName = gridName,
+                SpringConfigUrl = "config\\native-client-test-cache.xml",
+                JvmClasspath = TestUtils.CreateTestClasspath(),
+                PortableConfiguration = new PortableConfiguration
+                {
+                    TypeConfigurations = new List<PortableTypeConfiguration>
+                    {
+                        new PortableTypeConfiguration(typeof (CacheTestKey)),
+                        new PortableTypeConfiguration(typeof (TestReferenceObject)),
+                        new PortableTypeConfiguration(typeof (StreamReceiverPortable)),
+                        new PortableTypeConfiguration(typeof (EntryProcessorPortable)),
+                        new PortableTypeConfiguration(typeof (PortableEntry))
+                    }
+                },
+                JvmOptions = TestUtils.TestJavaOptions().Concat(new[]
+                {
+                    "-Xms3096m",
+                    "-Xmx3096m",
+                    "-XX:+UseParNewGC",
+                    "-XX:+UseConcMarkSweepGC",
+                    "-XX:+UseTLAB",
+                    "-XX:NewSize=128m",
+                    "-XX:MaxNewSize=128m",
+                    "-XX:MaxTenuringThreshold=0",
+                    "-XX:SurvivorRatio=1024",
+                    "-XX:+UseCMSInitiatingOccupancyOnly",
+                    "-XX:CMSInitiatingOccupancyFraction=60"
+                }).ToArray()
+            };
+        }
+
+        /// <summary>
+        /// Test portable receiver.
+        /// </summary>
+        private class StreamReceiverPortable : IStreamReceiver<int, int>
+        {
+            /** <inheritdoc /> */
+            public void Receive(ICache<int, int> cache, ICollection<ICacheEntry<int, int>> entries)
+            {
+                cache.PutAll(entries.ToDictionary(x => x.Key, x => x.Value + 1));
+            }
+        }
+
+        /// <summary>
+        /// Test portable receiver.
+        /// </summary>
+        [Serializable]
+        private class StreamReceiverKeepPortable : IStreamReceiver<int, IPortableObject>
+        {
+            /** <inheritdoc /> */
+            public void Receive(ICache<int, IPortableObject> cache, ICollection<ICacheEntry<int, IPortableObject>> entries)
+            {
+                var portables = cache.Ignite.Portables();
+
+                cache.PutAll(entries.ToDictionary(x => x.Key, x =>
+                    portables.ToPortable<IPortableObject>(new PortableEntry
+                    {
+                        Val = x.Value.Deserialize<PortableEntry>().Val + 1
+                    })));
+            }
+        }
+
+        /// <summary>
+        /// Test serializable receiver.
+        /// </summary>
+        [Serializable]
+        private class StreamReceiverSerializable : IStreamReceiver<int, int>
+        {
+            /** <inheritdoc /> */
+            public void Receive(ICache<int, int> cache, ICollection<ICacheEntry<int, int>> entries)
+            {
+                cache.PutAll(entries.ToDictionary(x => x.Key, x => x.Value + 1));
+            }
+        }
+
+        /// <summary>
+        /// Test entry processor.
+        /// </summary>
+        [Serializable]
+        private class EntryProcessorSerializable : ICacheEntryProcessor<int, int, int, int>
+        {
+            /** <inheritdoc /> */
+            public int Process(IMutableCacheEntry<int, int> entry, int arg)
+            {
+                entry.Value = entry.Key + 1;
+                
+                return 0;
+            }
+        }
+
+        /// <summary>
+        /// Test entry processor.
+        /// </summary>
+        private class EntryProcessorPortable : ICacheEntryProcessor<int, int, int, int>, IPortableMarshalAware
+        {
+            /** <inheritdoc /> */
+            public int Process(IMutableCacheEntry<int, int> entry, int arg)
+            {
+                entry.Value = entry.Key + 1;
+                
+                return 0;
+            }
+
+            /** <inheritdoc /> */
+            public void WritePortable(IPortableWriter writer)
+            {
+                // No-op.
+            }
+
+            /** <inheritdoc /> */
+            public void ReadPortable(IPortableReader reader)
+            {
+                // No-op.
+            }
+        }
+
+        /// <summary>
+        /// Portablecache entry.
+        /// </summary>
+        private class PortableEntry
+        {
+            public int Val { get; set; }
+        }
+    }
+}


[35/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/IClusterNode.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/IClusterNode.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/IClusterNode.cs
new file mode 100644
index 0000000..dfdccef
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/IClusterNode.cs
@@ -0,0 +1,138 @@
+/*
+ * 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.Cluster
+{
+    using System;
+    using System.Collections.Generic;
+
+    /// <summary>
+    /// Interface representing a single cluster node. Use <see cref="IClusterNode.Attribute{T}(string)"/> or
+    /// <see cref="IClusterNode.Metrics()"/> to get static and dynamic information about remote nodes.
+    /// You can get a list of all nodes in grid by calling <see cref="IClusterGroup.Nodes()"/> 
+    /// on <see cref="IIgnite"/> instance.
+    /// <para />
+    /// You can use Ignite node attributes to provide static information about a node.
+    /// This information is initialized once within grid, during node startup, and
+    /// remains the same throughout the lifetime of a node. 
+    /// <para/>
+    /// All members are thread-safe and may be used concurrently from multiple threads.
+    /// </summary>
+    public interface IClusterNode {
+        /// <summary>
+        /// Globally unique node ID. A new ID is generated every time a node restarts.
+        /// </summary>
+        Guid Id 
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Gets node's attribute. Attributes are assigned to nodes at startup.
+        /// <para />
+        /// Note that attributes cannot be changed at runtime.
+        /// </summary>
+        /// <param name="name">Attribute name.</param>
+        /// <returns>Attribute value.</returns>
+        T Attribute<T>(string name);
+        
+        /// <summary>
+        /// Try getting node's attribute. Attributes are assigned to nodes at startup.
+        /// <para />
+        /// Note that attributes cannot be changed at runtime.
+        /// </summary>
+        /// <param name="name">Attribute name.</param>
+        /// <param name="attr">Attribute value.</param>
+        /// <returns><code>true</code> in case such attribute exists.</returns>
+        bool TryGetAttribute<T>(string name, out T attr);
+
+        /// <summary>
+        /// Gets all node attributes. Attributes are assigned to nodes at startup.
+        /// <para />
+        /// Note that attributes cannot be changed at runtime.
+        /// </summary>
+        /// <returns>All node attributes.</returns>
+        IDictionary<string, object> Attributes();
+
+        /// <summary>
+        /// Collection of addresses this node is known by. 
+        /// </summary>
+        /// <returns>Collection of addresses.</returns>
+        ICollection<string> Addresses
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Collection of host names this node is known by.
+        /// </summary>
+        /// <returns>Collection of host names.</returns>
+        ICollection<string> HostNames
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Node order within grid topology. Discovery SPIs that support node ordering will
+        /// assign a proper order to each node and will guarantee that discovery event notifications
+        /// for new nodes will come in proper order. All other SPIs not supporting ordering
+        /// may choose to return node startup time here.
+        /// </summary>
+        long Order
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Tests whether or not this node is a local node.
+        /// </summary>
+        bool IsLocal
+        {
+            get;
+        }
+        
+        /// <summary>
+        /// Tests whether or not this node is a daemon.
+        /// <p/>
+        /// Daemon nodes are the usual Ignite nodes that participate in topology but not
+        /// visible on the main APIs, i.e. they are not part of any projections.
+        /// <p/>
+        /// Daemon nodes are used primarily for management and monitoring functionality that
+        /// is build on Ignite and needs to participate in the topology but should be
+        /// excluded from "normal" topology so that it won't participate in task execution
+        /// or in-memory database.
+        /// <p/>
+        /// Application code should never use daemon nodes.
+        /// </summary>
+        bool IsDaemon
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Gets metrics snapshot for this node. Note that node metrics are constantly updated
+        /// and provide up to date information about nodes. For example, you can get
+        /// an idea about CPU load on remote node via <see cref="IClusterMetrics.CurrentCpuLoad"/>.
+        /// <para/>
+        /// Node metrics are updated with some delay which is directly related to heartbeat
+        /// frequency. For example, when used with default <code>GridTcpDiscoverySpi</code> the 
+        /// update will happen every <code>2</code> seconds.
+        /// </summary>
+        /// <returns>Runtime metrics snapshot for this node.</returns>
+        IClusterMetrics Metrics();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/IClusterNodeFilter.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/IClusterNodeFilter.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/IClusterNodeFilter.cs
new file mode 100644
index 0000000..77eefbb
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/IClusterNodeFilter.cs
@@ -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.
+ */
+
+namespace Apache.Ignite.Core.Cluster
+{
+    /// <summary>
+    /// Represents cluster node filter.
+    /// </summary>
+    public interface IClusterNodeFilter
+    {
+        /// <summary>
+        /// Returns a value indicating whether provided node satisfies this predicate.
+        /// </summary>
+        /// <param name="node">Cluster node.</param>
+        /// <returns>Value indicating whether provided node satisfies this predicate.</returns>
+        bool Invoke(IClusterNode node);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/IAsyncSupport.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/IAsyncSupport.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/IAsyncSupport.cs
index f6b6551..ee98c5a 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/IAsyncSupport.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/IAsyncSupport.cs
@@ -18,7 +18,7 @@
 namespace Apache.Ignite.Core.Common
 {
     /// <summary>
-    /// Allows to enable asynchronous mode on Grid APIs.
+    /// Allows to enable asynchronous mode on Ignite APIs.
     /// </summary>
     /// <typeparam name="TWithAsync">Type of WithAsync method result.</typeparam>
     public interface IAsyncSupport<out TWithAsync> where TWithAsync : IAsyncSupport<TWithAsync>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/IgniteException.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/IgniteException.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/IgniteException.cs
index 4626407..98e5389 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/IgniteException.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/IgniteException.cs
@@ -21,7 +21,7 @@ namespace Apache.Ignite.Core.Common
     using System.Runtime.Serialization;
 
     /// <summary>
-    /// General grid exception. Indicates any error condition within Grid.
+    /// General Ignite exception. Indicates any error condition within Ignite.
     /// </summary>
     [Serializable]
     public class IgniteException : Exception

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/IgniteGuid.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/IgniteGuid.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/IgniteGuid.cs
new file mode 100644
index 0000000..53c7151
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/IgniteGuid.cs
@@ -0,0 +1,138 @@
+/*
+ * 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.Common
+{
+    using System;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Ignite guid with additional local ID.
+    /// </summary>
+    public struct IgniteGuid : IEquatable<IgniteGuid>
+    {
+        /** Global id. */
+        private readonly Guid _globalId;
+
+        /** Local id. */
+        private readonly long _localId;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="IgniteGuid"/> struct.
+        /// </summary>
+        /// <param name="globalId">The global id.</param>
+        /// <param name="localId">The local id.</param>
+        public IgniteGuid(Guid globalId, long localId)
+        {
+            _globalId = globalId;
+            _localId = localId;
+        }
+
+        /// <summary>
+        /// Gets the global id.
+        /// </summary>
+        public Guid GlobalId
+        {
+            get { return _globalId; }
+        }
+
+        /// <summary>
+        /// Gets the local id.
+        /// </summary>
+        public long LocalId
+        {
+            get { return _localId; }
+        }
+
+        /** <inheritDoc /> */
+        public bool Equals(IgniteGuid other)
+        {
+            return _globalId.Equals(other._globalId) && _localId == other._localId;
+        }
+
+        /** <inheritDoc /> */
+        public override bool Equals(object obj)
+        {
+            if (ReferenceEquals(null, obj)) return false;
+            return obj is IgniteGuid && Equals((IgniteGuid) obj);
+        }
+
+        /** <inheritDoc /> */
+        public override int GetHashCode()
+        {
+            unchecked
+            {
+                return (_globalId.GetHashCode() * 397) ^ _localId.GetHashCode();
+            }
+        }
+
+        /** <inheritDoc /> */
+        public override string ToString()
+        {
+            return string.Format("IgniteGuid [GlobalId={0}, LocalId={1}]", GlobalId, LocalId);
+        }
+
+        /// <summary>
+        /// Writes this object to the given writer.
+        /// </summary> 
+        /// <param name="w">Writer.</param>
+        public void WritePortable(IPortableRawWriter w)
+        {
+            w.WriteGuid(GlobalId);
+            w.WriteLong(LocalId);
+        }
+
+        /// <summary>
+        /// Reads this object from the given reader.
+        /// </summary> 
+        /// <param name="r">Reader.</param>
+        public static IgniteGuid ReadPortable(IPortableRawReader r)
+        {
+            var guid = r.ReadGuid();
+
+            return guid == null
+                ? new IgniteGuid(Guid.Empty, 0)
+                : new IgniteGuid(guid.Value, r.ReadLong());
+        }
+
+        /// <summary>
+        /// Implements the operator ==.
+        /// </summary>
+        /// <param name="a">First item.</param>
+        /// <param name="b">Second item.</param>
+        /// <returns>
+        /// The result of the operator.
+        /// </returns>
+        public static bool operator ==(IgniteGuid a, IgniteGuid b)
+        {
+            return a.Equals(b);
+        }
+
+        /// <summary>
+        /// Implements the operator !=.
+        /// </summary>
+        /// <param name="a">First item.</param>
+        /// <param name="b">Second item.</param>
+        /// <returns>
+        /// The result of the operator.
+        /// </returns>
+        public static bool operator !=(IgniteGuid a, IgniteGuid b)
+        {
+            return !(a == b);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeExecutionRejectedException.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeExecutionRejectedException.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeExecutionRejectedException.cs
new file mode 100644
index 0000000..108d396
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeExecutionRejectedException.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.Compute 
+{
+    using System;
+    using System.Runtime.Serialization;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// Indicates a situation when execution service provided by the user in configuration rejects execution.
+    /// </summary>
+    [Serializable]
+    public class ComputeExecutionRejectedException : IgniteException
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeExecutionRejectedException"/> class.
+        /// </summary>
+        public ComputeExecutionRejectedException()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeExecutionRejectedException" /> class.
+        /// </summary>
+        /// <param name="message">The message that describes the error.</param>
+        public ComputeExecutionRejectedException(string message)
+            : base(message)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeExecutionRejectedException"/> class.
+        /// </summary>
+        /// <param name="info">Serialization information.</param>
+        /// <param name="ctx">Streaming context.</param>
+        protected ComputeExecutionRejectedException(SerializationInfo info, StreamingContext ctx)
+            : base(info, ctx)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeExecutionRejectedException"/> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        public ComputeExecutionRejectedException(string message, Exception cause) : base(message, cause)
+        {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeJobAdapter.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeJobAdapter.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeJobAdapter.cs
new file mode 100644
index 0000000..92c6492
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeJobAdapter.cs
@@ -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.
+ */
+
+namespace Apache.Ignite.Core.Compute
+{
+    using System;
+
+    /// <summary>
+    /// Convenience adapter for <see cref="IComputeJob{T}"/> implementations. It provides the following functionality:
+    /// <ul>
+    /// <li>
+    ///      Default implementation of <see cref="IComputeJob{T}.Cancel()"/> method and ability
+    ///      to check whether cancellation occurred with <see cref="ComputeJobAdapter{T}.IsCancelled()"/> method.
+    /// </li>
+    /// <li>
+    ///      Ability to set and get job arguments via <see cref="ComputeJobAdapter{T}.SetArguments(object[])"/>
+    ///      and <see cref="ComputeJobAdapter{T}.Argument{T}(int)"/> methods.
+    /// </li>
+    /// </ul>
+    /// </summary>
+    [Serializable]
+    public abstract class ComputeJobAdapter<T> : IComputeJob<T>
+    {
+        /** Cancelled flag */
+        [NonSerialized]
+        private volatile bool _cancelled;
+
+        /** Arguments. */
+        protected object[] Args;
+
+        /// <summary>
+        /// No-arg constructor.
+        /// </summary>
+        protected ComputeJobAdapter()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Creates job with specified arguments.
+        /// </summary>
+        /// <param name="args">Optional job arguments.</param>
+        protected ComputeJobAdapter(params object[] args)
+        {
+            Args = args;
+        }
+
+        /// <summary>
+        /// This method is called when system detects that completion of this
+        /// job can no longer alter the overall outcome (for example, when parent task
+        /// has already reduced the results).
+        /// <para />
+        /// Note that job cancellation is only a hint, and it is really up to the actual job
+        /// instance to gracefully finish execution and exit.
+        /// </summary>
+        public void Cancel()
+        {
+            _cancelled = true;
+        }
+
+        /// <summary>
+        /// Sets given arguments.
+        /// </summary>
+        /// <param name="args">Optional job arguments to set.</param>
+        public void SetArguments(params object[] args)
+        {
+            Args = args;
+        }
+
+        /// <summary>
+        /// Sets given arguments.
+        /// </summary>
+        /// <param name="idx">Index of the argument.</param>
+        public TArg Argument<TArg>(int idx)
+        {
+            if (idx < 0 || idx >= Args.Length)
+                throw new ArgumentException("Invalid argument index: " + idx);
+
+            return (TArg)Args[idx];
+        }
+
+        /// <summary>
+        /// This method tests whether or not this job was cancelled. This method
+        /// is thread-safe and can be called without extra synchronization.
+        /// <p/>
+        /// This method can be periodically called in <see cref="IComputeJob{T}.Execute()"/> method
+        /// implementation to check whether or not this job cancelled. Note that system
+        /// calls <see cref="IComputeJob{T}.Cancel()"/> method only as a hint and this is a responsibility of
+        /// the implementation of the job to properly cancel its execution.
+        /// </summary>
+        /// <returns><c>True</c> if this job was cancelled, <c>false</c> otherwise.</returns>
+        protected bool IsCancelled()
+        {
+            return _cancelled;
+        }
+
+        /// <summary>
+        /// Executes this job.
+        /// </summary>
+        /// <returns>
+        /// Job execution result (possibly <c>null</c>). This result will be returned
+        /// in <see cref="IComputeJobResult{T}" /> object passed into
+        /// <see cref="IComputeTask{A,T,R}.Result" />
+        /// on caller node.
+        /// </returns>
+        public abstract T Execute();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeJobFailoverException.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeJobFailoverException.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeJobFailoverException.cs
new file mode 100644
index 0000000..970bd43
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeJobFailoverException.cs
@@ -0,0 +1,72 @@
+/*
+ * 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.Compute
+{
+    using System;
+    using System.Runtime.Serialization;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// This runtime exception can be thrown from <see cref="IComputeJob{T}.Execute()"/>
+    /// method to force job failover to another node within task topology.
+    /// <see cref="IComputeFunc{T,R}"/> or <see cref="IComputeFunc{T}"/>
+    /// passed into any of the <see cref="ICompute"/> methods can also throw this exception
+    /// to force failover.
+    /// </summary>
+    [Serializable]
+    public class ComputeJobFailoverException : IgniteException
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeJobFailoverException"/> class.
+        /// </summary>
+        public ComputeJobFailoverException()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeJobFailoverException" /> class.
+        /// </summary>
+        /// <param name="message">The message that describes the error.</param>
+        public ComputeJobFailoverException(string message) : base(message)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeJobFailoverException"/> class.
+        /// </summary>
+        /// <param name="info">Serialization information.</param>
+        /// <param name="ctx">Streaming context.</param>
+        protected ComputeJobFailoverException(SerializationInfo info, StreamingContext ctx)
+            : base(info, ctx)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeJobFailoverException"/> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        public ComputeJobFailoverException(string message, Exception cause) : base(message, cause)
+        {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeJobResultPolicy.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeJobResultPolicy.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeJobResultPolicy.cs
new file mode 100644
index 0000000..6fa0808
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeJobResultPolicy.cs
@@ -0,0 +1,45 @@
+/*
+ * 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.Compute
+{
+    using System.Collections.Generic;
+
+    /// <summary>
+    /// This enumeration provides different types of actions following the last received job result. See 
+    /// <see cref="IComputeTask{A,T,R}.Result(IComputeJobResult{T}, IList{IComputeJobResult{T}})"/>
+    /// for more details.
+    /// </summary>
+    public enum ComputeJobResultPolicy
+    {
+        /// <summary>
+        /// Wait for results if any are still expected. If all results have been received -
+        /// it will start reducing results.
+        /// </summary>
+        Wait = 0,
+
+        /// <summary>
+        /// Ignore all not yet received results and start reducing results.
+        /// </summary>
+        Reduce = 1,
+
+        /// <summary>
+        /// Fail-over job to execute on another node.
+        /// </summary>
+        Failover = 2
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeTaskAdapter.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeTaskAdapter.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeTaskAdapter.cs
new file mode 100644
index 0000000..67f7432
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeTaskAdapter.cs
@@ -0,0 +1,93 @@
+/*
+ * 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.Compute
+{
+    using System;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// Convenience adapter for <see cref="IComputeTask{A,T,R}"/> interface
+    /// </summary>
+    public abstract class ComputeTaskAdapter<TA, T, TR> : IComputeTask<TA, T, TR>
+    {
+        /// <summary>
+        /// Default implementation which will wait for all jobs to complete before
+        /// calling <see cref="IComputeTask{A,T,R}.Reduce"/> method.
+        /// <p/>
+        /// If remote job resulted in exception <see cref="IComputeJobResult{T}.Exception()"/> 
+        /// is not <c>null</c>),
+        /// then <see cref="ComputeJobResultPolicy.Failover"/>  policy will be returned if 
+        /// the exception is instance of <see cref="ClusterTopologyException"/> 
+        /// or <see cref="ComputeExecutionRejectedException"/>, which means that
+        /// remote node either failed or job execution was rejected before it got a chance to start. In all
+        /// other cases the exception will be rethrown which will ultimately cause task to fail.
+        /// </summary>
+        /// <param name="res">Received remote Ignite executable result.</param>
+        /// <param name="rcvd">All previously received results.</param>
+        /// <returns>Result policy that dictates how to process further upcoming job results.</returns>
+        public virtual ComputeJobResultPolicy Result(IComputeJobResult<T> res, IList<IComputeJobResult<T>> rcvd)
+        {
+            Exception err = res.Exception();
+
+            if (err != null)
+            {
+                if (err is ComputeExecutionRejectedException || err is ClusterTopologyException ||
+                    err is ComputeJobFailoverException)
+                    return ComputeJobResultPolicy.Failover;
+                
+                throw new IgniteException("Remote job threw user exception (override or implement IComputeTask.result(..) " +
+                                        "method if you would like to have automatic failover for this exception).", err);
+            }
+
+            return ComputeJobResultPolicy.Wait;
+        }
+
+        /// <summary>
+        /// This method is called to map or split Ignite task into multiple Ignite jobs. This is the
+        /// first method that gets called when task execution starts.
+        /// </summary>
+        /// <param name="subgrid">Nodes available for this task execution. Note that order of nodes is
+        /// guaranteed to be randomized by container. This ensures that every time you simply iterate
+        /// through Ignite nodes, the order of nodes will be random which over time should result into
+        /// all nodes being used equally.</param>
+        /// <param name="arg">Task execution argument. Can be <c>null</c>. This is the same argument
+        /// as the one passed into <c>ICompute.Execute()</c> methods.</param>
+        /// <returns>
+        /// Map of Ignite jobs assigned to subgrid node. If <c>null</c> or empty map is returned,
+        /// exception will be thrown.
+        /// </returns>
+        public abstract IDictionary<IComputeJob<T>, IClusterNode> Map(IList<IClusterNode> subgrid, TA arg);
+
+        /// <summary>
+        /// Reduces (or aggregates) results received so far into one compound result to be returned to
+        /// caller via future.
+        /// <para />
+        /// Note, that if some jobs did not succeed and could not be failed over then the list of
+        /// results passed into this method will include the failed results. Otherwise, failed
+        /// results will not be in the list.
+        /// </summary>
+        /// <param name="results">Received job results. Note that if task class has
+        /// <see cref="ComputeTaskNoResultCacheAttribute" /> attribute, then this list will be empty.</param>
+        /// <returns>
+        /// Task result constructed from results of remote executions.
+        /// </returns>
+        public abstract TR Reduce(IList<IComputeJobResult<T>> results);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeTaskCancelledException.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeTaskCancelledException.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeTaskCancelledException.cs
new file mode 100644
index 0000000..460e9b0
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeTaskCancelledException.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.Compute
+{
+    using System;
+    using System.Runtime.Serialization;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// This exception indicates that Ignite task was cancelled.
+    /// </summary>
+    [Serializable]
+    public class ComputeTaskCancelledException : IgniteException
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeTaskCancelledException"/> class.
+        /// </summary>
+        public ComputeTaskCancelledException()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeTaskCancelledException"/> class.
+        /// </summary>
+        /// <param name="message">The message that describes the error.</param>
+        public ComputeTaskCancelledException(string message)
+            : base(message)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeTaskCancelledException"/> class.
+        /// </summary>
+        /// <param name="info">Serialization information.</param>
+        /// <param name="ctx">Streaming context.</param>
+        protected ComputeTaskCancelledException(SerializationInfo info, StreamingContext ctx)
+            : base(info, ctx)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeTaskCancelledException"/> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        public ComputeTaskCancelledException(string message, Exception cause) : base(message, cause)
+        {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeTaskNoResultCacheAttribute.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeTaskNoResultCacheAttribute.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeTaskNoResultCacheAttribute.cs
new file mode 100644
index 0000000..a58aa87
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeTaskNoResultCacheAttribute.cs
@@ -0,0 +1,35 @@
+/*
+ * 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.Compute
+{
+    using System;
+
+    /// <summary>
+    /// This attribute disables caching of task results when attached to <see cref="IComputeTask{A,T,R}"/> 
+    /// instance. Use it when number of jobs within task grows too big, or jobs themselves are too large 
+    /// to keep in memory throughout task execution. By default all results are cached and passed into
+    /// <see cref="IComputeTask{A,T,R}.Result"/> 
+    /// and <see cref="IComputeTask{A,T,R}.Reduce"/> methods. When this 
+    /// attribute is attached to a task class, then this list of job results will always be empty.
+    /// </summary>
+    [AttributeUsage(AttributeTargets.Class | AttributeTargets.Interface)]
+    public sealed class ComputeTaskNoResultCacheAttribute : Attribute
+    {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeTaskSplitAdapter.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeTaskSplitAdapter.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeTaskSplitAdapter.cs
new file mode 100644
index 0000000..bf4685a
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeTaskSplitAdapter.cs
@@ -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.
+ */
+
+namespace Apache.Ignite.Core.Compute
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Compute;
+
+    /// <summary>
+    /// This class defines simplified adapter for <see cref="IComputeTask{A,T,R}"/>. This adapter can be used
+    /// when jobs can be randomly assigned to available Ignite nodes. This adapter is sufficient
+    /// in most homogeneous environments where all nodes are equally suitable for executing grid
+    /// job, see <see cref="Split"/> method for more details.
+    /// </summary>
+    public abstract class ComputeTaskSplitAdapter<TA, T, TR> : ComputeTaskAdapter<TA, T, TR>
+    {
+        /** Random generator */
+        [ThreadStatic]
+        // ReSharper disable once StaticMemberInGenericType
+        private static Random _rnd;
+
+        /// <summary>
+        /// This is a simplified version of <see cref="IComputeTask{A,T,R}.Map"/> method.
+        /// <p/>
+        /// This method basically takes given argument and splits it into a collection
+        /// of <see cref="IComputeJob"/> using provided grid size as indication of how many node are
+        /// available. These jobs will be randomly mapped to available Ignite nodes. Note that
+        /// if number of jobs is greater than number of Ignite nodes (i.e, grid size), the grid
+        /// nodes will be reused and some jobs will end up on the same Ignite nodes.
+        /// </summary>
+        /// <param name="gridSize">Number of available Ignite nodes. Note that returned number of jobs can be less, 
+        ///  equal or greater than this grid size.</param>
+        /// <param name="arg">Task execution argument. Can be <c>null</c>.</param>
+        protected abstract ICollection<IComputeJob<T>> Split(int gridSize, TA arg);
+
+        /// <summary>
+        /// This method is called to map or split Ignite task into multiple Ignite jobs. This is the
+        /// first method that gets called when task execution starts.
+        /// </summary>
+        /// <param name="subgrid">Nodes available for this task execution. Note that order of nodes is
+        /// guaranteed to be randomized by container. This ensures that every time you simply iterate
+        /// through Ignite nodes, the order of nodes will be random which over time should result into
+        /// all nodes being used equally.</param>
+        /// <param name="arg">Task execution argument. Can be <c>null</c>. This is the same argument
+        /// as the one passed into <c>ICompute.Execute()</c> methods.</param>
+        /// <returns>
+        /// Map of Ignite jobs assigned to subgrid node. If <c>null</c> or empty map is returned,
+        /// exception will be thrown.
+        /// </returns>
+        /// <exception cref="IgniteException">Split returned no jobs.</exception>
+        override public IDictionary<IComputeJob<T>, IClusterNode> Map(IList<IClusterNode> subgrid, TA arg)
+        {
+            Debug.Assert(subgrid != null && subgrid.Count > 0);
+
+            var jobs = Split(subgrid.Count, arg);
+
+            if (jobs == null || jobs.Count == 0)
+                throw new IgniteException("Split returned no jobs.");
+
+            var map = new Dictionary<IComputeJob<T>, IClusterNode>(jobs.Count);
+
+            if (_rnd == null)
+                _rnd = new Random();
+
+            foreach (var job in jobs)
+            {
+                int idx = _rnd.Next(subgrid.Count);
+
+                IClusterNode node = subgrid[idx];
+
+                map[job] = node;
+            }
+
+            return map;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeTaskTimeoutException.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeTaskTimeoutException.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeTaskTimeoutException.cs
new file mode 100644
index 0000000..71fc568
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeTaskTimeoutException.cs
@@ -0,0 +1,67 @@
+/*
+ * 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.Compute 
+{
+    using System;
+    using System.Runtime.Serialization;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// Indicates that task execution timed out.
+    /// </summary>
+    [Serializable]
+    public class ComputeTaskTimeoutException : IgniteException
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeTaskTimeoutException"/> class.
+        /// </summary>
+        public ComputeTaskTimeoutException()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeTaskTimeoutException"/> class.
+        /// </summary>
+        /// <param name="message">The message that describes the error.</param>
+        public ComputeTaskTimeoutException(string message) : base(message)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeTaskTimeoutException"/> class.
+        /// </summary>
+        /// <param name="info">Serialization information.</param>
+        /// <param name="ctx">Streaming context.</param>
+        protected ComputeTaskTimeoutException(SerializationInfo info, StreamingContext ctx) : base(info, ctx)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeTaskTimeoutException"/> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        public ComputeTaskTimeoutException(string message, Exception cause) : base(message, cause)
+        {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeUserUndeclaredException.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeUserUndeclaredException.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeUserUndeclaredException.cs
new file mode 100644
index 0000000..e3c090e
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ComputeUserUndeclaredException.cs
@@ -0,0 +1,70 @@
+/*
+ * 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.Compute 
+{
+    using System;
+    using System.Runtime.Serialization;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// This exception is thrown when user's code throws undeclared runtime exception. By user core it is
+    /// assumed the code in Ignite task, Ignite job or SPI. In most cases it should be an indication of unrecoverable
+    /// error condition such as assertion, out of memory error, etc.
+    /// </summary>
+    [Serializable]
+    public class ComputeUserUndeclaredException : IgniteException
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeUserUndeclaredException"/> class.
+        /// </summary>
+        public ComputeUserUndeclaredException()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeUserUndeclaredException"/> class.
+        /// </summary>
+        /// <param name="message">The message that describes the error.</param>
+        public ComputeUserUndeclaredException(string message) : base(message)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeUserUndeclaredException"/> class.
+        /// </summary>
+        /// <param name="info">Serialization information.</param>
+        /// <param name="ctx">Streaming context.</param>
+        protected ComputeUserUndeclaredException(SerializationInfo info, StreamingContext ctx)
+            : base(info, ctx)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeUserUndeclaredException"/> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        public ComputeUserUndeclaredException(string message, Exception cause) : base(message, cause)
+        {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ICompute.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ICompute.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ICompute.cs
new file mode 100644
index 0000000..bbb496f
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/ICompute.cs
@@ -0,0 +1,274 @@
+/*
+ * 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.Compute
+{
+    using System;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// Defines Ignite functionality for executing tasks and closures over nodes
+    /// in the <see cref="IClusterGroup"/>. Instance of <see cref="ICompute"/>
+    /// is obtained from grid projection using <see cref="IClusterGroup.Compute()"/> method.
+    /// <para />
+    /// Note that if attempt is made to execute a computation over an empty projection (i.e. projection that does
+    /// not have any alive nodes), <c>ClusterGroupEmptyException</c> will be thrown out of result future.
+    /// <para />
+    /// Ignite must select a node for a computation to be executed. The node will be selected based on the
+    /// underlying <c>GridLoadBalancingSpi</c>, which by default sequentially picks next available node from
+    /// grid projection. Other load balancing policies, such as <c>random</c> or <c>adaptive</c>, can be
+    /// configured as well by selecting different load balancing SPI in Ignite configuration. If your logic requires
+    /// some custom load balancing behavior, consider implementing <c>ComputeTask</c> in Java directly.
+    /// <para />
+    /// Ignite guarantees that as long as there is at least one Ignite node standing, every job will be
+    /// executed. Jobs will automatically failover to another node if a remote node crashed or has rejected
+    /// execution due to lack of resources. By default, in case of failover, next load balanced node will be
+    /// picked for job execution. Also jobs will never be re-routed to the nodes they have failed on. This
+    /// behavior can be changed by configuring any of the existing or a custom <c>FailoverSpi</c> in Ignite
+    /// configuration.
+    /// <para/>
+    /// All members are thread-safe and may be used concurrently from multiple threads.
+    /// </summary>
+    public interface ICompute : IAsyncSupport<ICompute>
+    {
+        /// <summary>
+        /// Grid projection to which this compute instance belongs.
+        /// </summary>
+        IClusterGroup ClusterGroup
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Sets no-failover flag for the next executed task on this projection in the current thread.
+        /// If flag is set, job will be never failed over even if remote node crashes or rejects execution.
+        /// When task starts execution, the no-failover flag is reset, so all other task will use default
+        /// failover policy, unless this flag is set again.
+        /// </summary>
+        /// <returns>This compute instance for chaining calls.</returns>
+        ICompute WithNoFailover();
+
+        /// <summary>
+        /// Sets task timeout for the next executed task on this projection in the current thread.
+        /// When task starts execution, the timeout is reset, so one timeout is used only once.
+        /// </summary>
+        /// <param name="timeout">Computation timeout in milliseconds.</param>
+        /// <returns>This compute instance for chaining calls.</returns>
+        ICompute WithTimeout(long timeout);
+
+        /// <summary>
+        /// Sets keep-portable flag for the next executed Java task on this projection in the current
+        /// thread so that task argument passed to Java and returned task results will not be
+        /// deserialized.
+        /// </summary>
+        /// <returns>This compute instance for chaining calls.</returns>
+        ICompute WithKeepPortable();
+
+        /// <summary>
+        /// Executes given Java task on the grid projection. If task for given name has not been deployed yet,
+        /// then 'taskName' will be used as task class name to auto-deploy the task.
+        /// </summary>
+        /// <param name="taskName">Java task name</param>
+        /// <param name="taskArg">Optional argument of task execution, can be null.</param>
+        /// <returns>Task result.</returns>
+        /// <typeparam name="T">Type of task result.</typeparam>
+        T ExecuteJavaTask<T>(string taskName, object taskArg);
+
+        /// <summary>
+        /// Executes given task on the grid projection. For step-by-step explanation of task execution process
+        /// refer to <see cref="IComputeTask{A,T,R}"/> documentation.
+        /// </summary>
+        /// <param name="task">Task to execute.</param>
+        /// <param name="taskArg">Optional task argument.</param>
+        /// <returns>Task result.</returns>
+        /// <typeparam name="TA">Argument type.</typeparam>
+        /// <typeparam name="T">Type of job result.</typeparam>
+        /// <typeparam name="TR">Type of reduce result.</typeparam>
+        [AsyncSupported]
+        TR Execute<TA, T, TR>(IComputeTask<TA, T, TR> task, TA taskArg);
+        
+        /// <summary>
+        /// Executes given task on the grid projection. For step-by-step explanation of task execution process
+        /// refer to <see cref="IComputeTask{A,T,R}"/> documentation.
+        /// </summary>
+        /// <param name="task">Task to execute.</param>
+        /// <returns>Task result.</returns>
+        /// <typeparam name="T">Type of job result.</typeparam>
+        /// <typeparam name="TR">Type of reduce result.</typeparam>
+        [AsyncSupported]
+        TR Execute<T, TR>(IComputeTask<T, TR> task);
+
+        /// <summary>
+        /// Executes given task on the grid projection. For step-by-step explanation of task execution process
+        /// refer to <see cref="IComputeTask{A,T,R}"/> documentation.
+        /// </summary>
+        /// <param name="taskType">Task type.</param>
+        /// <param name="taskArg">Optional task argument.</param>
+        /// <returns>Task result.</returns>
+        /// <typeparam name="TA">Argument type.</typeparam>
+        /// <typeparam name="T">Type of job result.</typeparam>
+        /// <typeparam name="TR">Type of reduce result.</typeparam>
+        [AsyncSupported]
+        TR Execute<TA, T, TR>(Type taskType, TA taskArg);
+        
+        /// <summary>
+        /// Executes given task on the grid projection. For step-by-step explanation of task execution process
+        /// refer to <see cref="IComputeTask{A,T,R}"/> documentation.
+        /// </summary>
+        /// <param name="taskType">Task type.</param>
+        /// <returns>Task result.</returns>
+        /// <typeparam name="T">Type of job result.</typeparam>
+        /// <typeparam name="TR">Type of reduce result.</typeparam>
+        [AsyncSupported]
+        TR Execute<T, TR>(Type taskType);
+
+        /// <summary>
+        /// Executes provided job on a node in this grid projection. The result of the
+        /// job execution is returned from the result closure.
+        /// </summary>
+        /// <param name="clo">Job to execute.</param>
+        /// <returns>Job result for this execution.</returns>
+        /// <typeparam name="TR">Type of job result.</typeparam>
+        [AsyncSupported]
+        TR Call<TR>(IComputeFunc<TR> clo);
+
+        /// <summary>
+        /// Executes given job on the node where data for provided affinity key is located 
+        /// (a.k.a. affinity co-location).
+        /// </summary>
+        /// <param name="cacheName">Name of the cache to use for affinity co-location.</param>
+        /// <param name="affinityKey">Affinity key.</param>
+        /// <param name="clo">Job to execute.</param>
+        /// <returns>Job result for this execution.</returns>
+        /// <typeparam name="TR">Type of job result.</typeparam>
+        [AsyncSupported]
+        TR AffinityCall<TR>(string cacheName, object affinityKey, IComputeFunc<TR> clo);
+
+        /// <summary>
+        /// Executes collection of jobs on nodes within this grid projection.
+        /// </summary>
+        /// <param name="clos">Collection of jobs to execute.</param>
+        /// <param name="rdc">Reducer to reduce all job results into one individual return value.</param>
+        /// <returns>Reduced job result for this execution.</returns>
+        /// <typeparam name="TR1">Type of job result.</typeparam>
+        /// <typeparam name="TR2">Type of reduced result.</typeparam>
+        [AsyncSupported]
+        TR2 Call<TR1, TR2>(IEnumerable<IComputeFunc<TR1>> clos, IComputeReducer<TR1, TR2> rdc);
+        
+        /// <summary>
+        /// Executes collection of jobs on nodes within this grid projection.
+        /// </summary>
+        /// <param name="clos">Collection of jobs to execute.</param>
+        /// <returns>Collection of job results for this execution.</returns>
+        /// <typeparam name="TR">Type of job result.</typeparam>
+        [AsyncSupported]
+        ICollection<TR> Call<TR>(IEnumerable<IComputeFunc<TR>> clos);
+
+        /// <summary>
+        /// Broadcasts given job to all nodes in grid projection. Every participating node will return a job result. 
+        /// </summary>
+        /// <param name="clo">Job to broadcast to all projection nodes.</param>
+        /// <returns>Collection of results for this execution.</returns>
+        [AsyncSupported]
+        ICollection<TR> Broadcast<TR>(IComputeFunc<TR> clo);
+
+        /// <summary>
+        /// Broadcasts given closure job with passed in argument to all nodes in grid projection.
+        /// Every participating node will return a job result.
+        /// </summary>
+        /// <param name="clo">Job to broadcast to all projection nodes.</param>
+        /// <param name="arg">Job closure argument.</param>
+        /// <returns>Collection of results for this execution.</returns>
+        /// <typeparam name="T">Type of argument.</typeparam>
+        /// <typeparam name="TR">Type of job result.</typeparam>
+        [AsyncSupported]
+        ICollection<TR> Broadcast<T, TR>(IComputeFunc<T, TR> clo, T arg);
+
+        /// <summary>
+        /// Broadcasts given job to all nodes in grid projection.
+        /// </summary>
+        /// <param name="action">Job to broadcast to all projection nodes.</param>
+        [AsyncSupported]
+        void Broadcast(IComputeAction action);
+
+        /// <summary>
+        /// Executes provided job on a node in this grid projection.
+        /// </summary>
+        /// <param name="action">Job to execute.</param>
+        [AsyncSupported]
+        void Run(IComputeAction action);
+
+        /// <summary>
+        /// Executes given job on the node where data for provided affinity key is located
+        /// (a.k.a. affinity co-location).
+        /// </summary>
+        /// <param name="cacheName">Name of the cache to use for affinity co-location.</param>
+        /// <param name="affinityKey">Affinity key.</param>
+        /// <param name="action">Job to execute.</param>
+        [AsyncSupported]
+        void AffinityRun(string cacheName, object affinityKey, IComputeAction action);
+
+        /// <summary>
+        /// Executes collection of jobs on Ignite nodes within this grid projection.
+        /// </summary>
+        /// <param name="actions">Jobs to execute.</param>
+        [AsyncSupported]
+        void Run(IEnumerable<IComputeAction> actions);
+
+        /// <summary>
+        /// Executes provided closure job on a node in this grid projection.
+        /// </summary>
+        /// <param name="clo">Job to run.</param>
+        /// <param name="arg">Job argument.</param>
+        /// <returns>Job result for this execution.</returns>
+        /// <typeparam name="T">Type of argument.</typeparam>
+        /// <typeparam name="TR">Type of job result.</typeparam>
+        [AsyncSupported]
+        TR Apply<T, TR>(IComputeFunc<T, TR> clo, T arg);
+
+        /// <summary>
+        /// Executes provided closure job on nodes within this grid projection. A new job is executed for
+        /// every argument in the passed in collection. The number of actual job executions will be
+        /// equal to size of the job arguments collection.
+        /// </summary>
+        /// <param name="clo">Job to run.</param>
+        /// <param name="args">Job arguments.</param>
+        /// <returns>Сollection of job results.</returns>
+        /// <typeparam name="T">Type of argument.</typeparam>
+        /// <typeparam name="TR">Type of job result.</typeparam>
+        [AsyncSupported]
+        ICollection<TR> Apply<T, TR>(IComputeFunc<T, TR> clo, IEnumerable<T> args);
+
+        /// <summary>
+        /// Executes provided closure job on nodes within this grid projection. A new job is executed for
+        /// every argument in the passed in collection. The number of actual job executions will be
+        /// equal to size of the job arguments collection. The returned job results will be reduced
+        /// into an individual result by provided reducer.
+        /// </summary>
+        /// <param name="clo">Job to run.</param>
+        /// <param name="args">Job arguments.</param>
+        /// <param name="rdc">Reducer to reduce all job results into one individual return value.</param>
+        /// <returns>Reduced job result for this execution.</returns>
+        /// <typeparam name="T">Type of argument.</typeparam>
+        /// <typeparam name="TR1">Type of job result.</typeparam>
+        /// <typeparam name="TR2">Type of reduced result.</typeparam>
+        [AsyncSupported]
+        TR2 Apply<T, TR1, TR2>(IComputeFunc<T, TR1> clo, IEnumerable<T> args, IComputeReducer<TR1, TR2> rdc);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/IComputeFunc.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/IComputeFunc.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/IComputeFunc.cs
new file mode 100644
index 0000000..4a43f11
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/IComputeFunc.cs
@@ -0,0 +1,55 @@
+/*
+ * 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.Compute
+{
+    /// <summary>
+    /// Defines function having a single argument.
+    /// </summary>
+    public interface IComputeFunc<in T, out TR>
+    {
+        /// <summary>
+        /// Invoke function.
+        /// </summary>
+        /// <param name="arg">Argument.</param>
+        /// <returns>Result.</returns>
+        TR Invoke(T arg);
+    }
+
+    /// <summary>
+    /// Defines function having no arguments.
+    /// </summary>
+    public interface IComputeFunc<out T>
+    {
+        /// <summary>
+        /// Invoke function.
+        /// </summary>
+        /// <returns>Result.</returns>
+        T Invoke();
+    }
+
+    /// <summary>
+    /// Defines a void function having no arguments.
+    /// </summary>
+    public interface IComputeAction
+    {
+        /// <summary>
+        /// Invokes action.
+        /// </summary>
+        void Invoke();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/IComputeJob.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/IComputeJob.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/IComputeJob.cs
new file mode 100644
index 0000000..3b8ac60
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/IComputeJob.cs
@@ -0,0 +1,58 @@
+/*
+ * 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.Compute
+{
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Resource;
+
+    /// <summary>
+    /// Defines executable unit for <see cref="IComputeTask{A,T,R}"/>. Ignite task gets split into jobs
+    /// when <see cref="IComputeTask{A,T,R}.Map(IList{IClusterNode}, A)"/> method is called. This
+    /// method returns all jobs for the task mapped to their corresponding Ignite nodes for execution. 
+    /// Grid will then serialize this jobs and send them to requested nodes for execution.
+    /// <para />
+    /// Once job execution is complete, the return value will be sent back to parent task and will 
+    /// be passed into 
+    /// <see cref="IComputeTask{A,T,R}.Result(IComputeJobResult{T}, IList{IComputeJobResult{T}})"/>
+    /// method via <see cref="IComputeJobResult{T}"/> instance. 
+    /// <para />
+    /// Ignite job implementation can be injected with <see cref="IIgnite"/> using 
+    /// <see cref="InstanceResourceAttribute"/> attribute.
+    /// </summary>
+    public interface IComputeJob<out T>
+    {
+        /// <summary>
+        /// Executes this job.
+        /// </summary>
+        /// <returns>Job execution result (possibly <c>null</c>). This result will be returned
+        /// in <see cref="IComputeJobResult{T}"/> object passed into 
+        /// <see cref="IComputeTask{A,T,R}.Result(IComputeJobResult{T}, IList{IComputeJobResult{T}})"/>
+        /// on caller node.</returns>
+        T Execute();
+
+        /// <summary>
+        /// This method is called when system detects that completion of this
+        /// job can no longer alter the overall outcome (for example, when parent task
+        /// has already reduced the results). 
+        /// <para />
+        /// Note that job cancellation is only a hint, and it is really up to the actual job
+        /// instance to gracefully finish execution and exit.
+        /// </summary>
+        void Cancel();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/IComputeJobResult.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/IComputeJobResult.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/IComputeJobResult.cs
new file mode 100644
index 0000000..5891fd7
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/IComputeJobResult.cs
@@ -0,0 +1,73 @@
+/*
+ * 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.Compute
+{
+    using System;
+    using System.Collections.Generic;
+
+    /// <summary>
+    /// Job execution result which gets passed to 
+    /// <see cref="IComputeTask{A,T,R}.Result(IComputeJobResult{T}, IList{IComputeJobResult{T}})"/>
+    /// method.
+    /// </summary>
+    public interface IComputeJobResult<out T>
+    {
+        /// <summary>
+        /// Gets data returned by remote job if it didn't fail. This data is the
+        /// object returned from <see cref="IComputeJob{T}.Execute()"/> method.
+        /// <para />
+        /// Note that if task is annotated with <see cref="ComputeTaskNoResultCacheAttribute"/> 
+        /// attribute, then job results will not be cached and will be available only in
+        /// <see cref="IComputeTask{A,T,R}.Result(IComputeJobResult{T}, IList{IComputeJobResult{T}})"/>
+        /// method for every individual job, but not in 
+        /// <see cref="IComputeTask{A,T,R}.Reduce(IList{IComputeJobResult{T}})"/> method.
+        /// 
+        /// </summary>
+        /// <returns>Data returned by job.</returns>
+        T Data();
+
+        /// <summary>
+        /// Gets local instance of remote job produced this result.
+        /// </summary>
+        /// <returns></returns>
+        IComputeJob<T> Job();
+
+        /// <summary>
+        /// Gets exception produced by execution of remote job, or <c>null</c> if no
+        /// exception was produced.
+        /// </summary>
+        /// <returns>Exception or <c>null</c> in case of success.</returns>
+        Exception Exception();
+
+        /// <summary>
+        /// ID of the node where actual job execution occurred.
+        /// </summary>
+        Guid NodeId
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Whether the job was cancelled.
+        /// </summary>
+        bool Cancelled
+        {
+            get;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/IComputeReducer.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/IComputeReducer.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/IComputeReducer.cs
new file mode 100644
index 0000000..46dcbd9
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/IComputeReducer.cs
@@ -0,0 +1,39 @@
+/*
+ * 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.Compute
+{
+    /// <summary>
+    /// Compute reducer which is capable of result collecting and reducing.
+    /// </summary>
+    public interface IComputeReducer<in TR1, out TR2>
+    {
+        /// <summary>
+        /// Collect closure execution result.
+        /// </summary>
+        /// <param name="res">Result.</param>
+        /// <returns><c>True</c> to continue collecting results until all closures are finished, 
+        /// <c>false</c> to start reducing.</returns>
+        bool Collect(TR1 res);
+
+        /// <summary>
+        /// Reduce closure execution results collected earlier.
+        /// </summary>
+        /// <returns>Reduce result.</returns>
+        TR2 Reduce();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/IComputeTask.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/IComputeTask.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/IComputeTask.cs
new file mode 100644
index 0000000..21b6c48
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Compute/IComputeTask.cs
@@ -0,0 +1,132 @@
+/*
+ * 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.Compute
+{
+    using System.Collections.Generic;
+    using System.Diagnostics.CodeAnalysis;
+    using Apache.Ignite.Core.Cluster;
+
+    /// <summary>
+    /// Ignite task interface defines a task that can be executed on the grid. Ignite task
+    /// is responsible for splitting business logic into multiple Ignite jobs, receiving
+    /// results from individual Ignite jobs executing on remote nodes, and reducing
+    /// (aggregating) received jobs' results into final Ignite task result.
+    /// <para />
+    /// Upon request to execute a task, the system will do the following:
+    /// <list type="bullet">
+    ///     <item>
+    ///         <description>Inject annotated resources into task instance.</description>
+    ///     </item>
+    ///     <item>
+    ///         <description>Apply <see cref="IComputeTask{A,T,R}.Map(IList{IClusterNode}, TA)"/>.
+    ///         This method is responsible for splitting business logic into multiple jobs 
+    ///         (units of execution) and mapping them to Ignite nodes.</description>
+    ///     </item>
+    ///     <item>
+    ///         <description>System will send mapped Ignite jobs to their respective nodes.</description>
+    ///     </item>
+    ///     <item>
+    ///         <description>Once job execution results become available method 
+    ///         <see cref="IComputeTask{A,T,R}.Result(IComputeJobResult{T}, IList{IComputeJobResult{T}})"/>
+    ///         will be called for ech received job result. The policy returned by this method will
+    ///         determine the way task reacts to every job result.
+    ///         <para />
+    ///         If <see cref="ComputeJobResultPolicy.Wait"/> is returned, task will continue to wait
+    ///         for other job results. If this result is the last job result, then reduce phase will be
+    ///         started.
+    ///         <para />
+    ///         If <see cref="ComputeJobResultPolicy.Reduce"/> is returned, reduce phase will be started
+    ///         right away without waiting for other jobs completion (all remaining jobs will receive cancel 
+    ///         request).
+    ///         <para />
+    ///         If <see cref="ComputeJobResultPolicy.Failover"/> is returned, job will be failed over to 
+    ///         another node for execution. Note that if you use <see cref="ComputeTaskAdapter{A,T,R}"/>, it will
+    ///         automatically fail jobs to another node for 2 well-known failure cases: 1) job has failed to due
+    ///         to node crash (in this case <see cref="IComputeJobResult{T}.Exception()"/> will return 
+    ///         <see cref="ClusterTopologyException"/>); 2) job execution was rejected, i.e. remote node 
+    ///         has cancelled job before it got a chance to execute, while it still was on the waiting list. 
+    ///         (in this case <see cref="IComputeJobResult{T}.Exception()"/> will return 
+    ///         <see cref="ComputeExecutionRejectedException"/>).
+    ///         </description>
+    ///     </item>
+    ///     <item>
+    ///         <description>Once all results are received or 
+    ///         <see cref="IComputeTask{A,T,R}.Result(IComputeJobResult{T}, IList{IComputeJobResult{T}})"/>
+    ///         method returned <see cref="ComputeJobResultPolicy.Reduce"/> policy, method 
+    ///         <see cref="IComputeTask{A,T,R}.Reduce(IList{IComputeJobResult{T}})"/>
+    ///         is called to aggregate received results into one final result. Once this method is finished the 
+    ///         execution of the Ignite task is complete. This result will be returned to the user through future.
+    ///         </description>    
+    ///     </item>
+    /// </list>
+    /// </summary>
+    /// <typeparam name="TA">Argument type.</typeparam>
+    /// <typeparam name="T">Type of job result.</typeparam>
+    /// <typeparam name="TR">Type of reduce result.</typeparam>
+    public interface IComputeTask<in TA, T, out TR>
+    {
+        /// <summary>
+        /// This method is called to map or split Ignite task into multiple Ignite jobs. This is the
+        /// first method that gets called when task execution starts.
+        /// </summary>
+        /// <param name="subgrid">Nodes available for this task execution. Note that order of nodes is
+        /// guaranteed to be randomized by container. This ensures that every time you simply iterate 
+        /// through Ignite nodes, the order of nodes will be random which over time should result into 
+        /// all nodes being used equally.</param>
+        /// <param name="arg">Task execution argument. Can be <c>null</c>. This is the same argument
+        /// as the one passed into <c>ICompute.Execute()</c> methods.</param>
+        /// <returns>Map of Ignite jobs assigned to subgrid node. If <c>null</c> or empty map is returned,
+        /// exception will be thrown.</returns>
+        IDictionary<IComputeJob<T>, IClusterNode> Map(IList<IClusterNode> subgrid, TA arg);
+
+        /// <summary>
+        /// Asynchronous callback invoked every time a result from remote execution is
+        /// received. It is ultimately upto this method to return a policy based
+        /// on which the system will either wait for more results, reduce results
+        /// received so far, or failover this job to another node. See 
+        /// <see cref="ComputeJobResultPolicy"/> for more information.
+        /// </summary>
+        /// <param name="res">Received remote Ignite executable result.</param>
+        /// <param name="rcvd">All previously received results. Note that if task class has
+        /// <see cref="ComputeTaskNoResultCacheAttribute"/> attribute, then this list will be empty.</param>
+        /// <returns>Result policy that dictates how to process further upcoming job results.</returns>
+        ComputeJobResultPolicy Result(IComputeJobResult<T> res, IList<IComputeJobResult<T>> rcvd);
+
+        /// <summary>
+        /// Reduces (or aggregates) results received so far into one compound result to be returned to 
+        /// caller via future.
+        /// <para />
+        /// Note, that if some jobs did not succeed and could not be failed over then the list of
+        /// results passed into this method will include the failed results. Otherwise, failed
+        /// results will not be in the list.
+        /// </summary>
+        /// <param name="results">Received job results. Note that if task class has 
+        /// <see cref="ComputeTaskNoResultCacheAttribute"/> attribute, then this list will be empty.</param>
+        /// <returns>Task result constructed from results of remote executions.</returns>
+        TR Reduce(IList<IComputeJobResult<T>> results);
+    }
+
+    /// <summary>
+    /// IComputeTask without an argument.
+    /// </summary>
+    [SuppressMessage("Microsoft.Design", "CA1040:AvoidEmptyInterfaces")]
+    public interface IComputeTask<T, out TR> : IComputeTask<object, T, TR>
+    {
+        // No-op.
+    }
+}


[13/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAffinityTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAffinityTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAffinityTest.cs
new file mode 100644
index 0000000..beb2c0f
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAffinityTest.cs
@@ -0,0 +1,139 @@
+/*
+ * 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.Cache
+{
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.Portable;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Affinity key tests.
+    /// </summary>
+    public class CacheAffinityTest
+    {
+        /// <summary>
+        ///
+        /// </summary>
+        [TestFixtureSetUp]
+        public virtual void StartGrids()
+        {
+            TestUtils.KillProcesses();
+
+            IgniteConfigurationEx cfg = new IgniteConfigurationEx();
+
+            cfg.JvmClasspath = TestUtils.CreateTestClasspath();
+            cfg.JvmOptions = TestUtils.TestJavaOptions();
+            cfg.SpringConfigUrl = "config\\native-client-test-cache-affinity.xml";
+
+            for (int i = 0; i < 3; i++)
+            {
+                cfg.GridName = "grid-" + i;
+
+                Ignition.Start(cfg);
+            }
+        }
+
+        /// <summary>
+        /// Tear-down routine.
+        /// </summary>
+        [TestFixtureTearDown]
+        public virtual void StopGrids()
+        {
+            for (int i = 0; i < 3; i++)
+                Ignition.Stop("grid-" + i, true);
+        }
+
+        /// <summary>
+        /// Test affinity key.
+        /// </summary>
+        [Test]
+        public void TestAffinity()
+        {
+            IIgnite g = Ignition.GetIgnite("grid-0");
+
+            ICacheAffinity aff = g.Affinity(null);
+
+            IClusterNode node = aff.MapKeyToNode(new AffinityTestKey(0, 1));
+
+            for (int i = 0; i < 10; i++)
+                Assert.AreEqual(node.Id, aff.MapKeyToNode(new AffinityTestKey(i, 1)).Id);
+        }
+
+        /// <summary>
+        /// Test affinity with portable flag.
+        /// </summary>
+        [Test]
+        public void TestAffinityPortable()
+        {
+            IIgnite g = Ignition.GetIgnite("grid-0");
+
+            ICacheAffinity aff = g.Affinity(null);  
+
+            IPortableObject affKey = g.Portables().ToPortable<IPortableObject>(new AffinityTestKey(0, 1));
+
+            IClusterNode node = aff.MapKeyToNode(affKey);
+
+            for (int i = 0; i < 10; i++)
+            {
+                IPortableObject otherAffKey =
+                    g.Portables().ToPortable<IPortableObject>(new AffinityTestKey(i, 1));
+
+                Assert.AreEqual(node.Id, aff.MapKeyToNode(otherAffKey).Id);
+            }
+        }
+
+        /// <summary>
+        /// Affinity key.
+        /// </summary>
+        public class AffinityTestKey
+        {
+            /** ID. */
+            private int _id;
+
+            /** Affinity key. */
+            private int _affKey;
+
+            /// <summary>
+            /// Constructor.
+            /// </summary>
+            /// <param name="id">ID.</param>
+            /// <param name="affKey">Affinity key.</param>
+            public AffinityTestKey(int id, int affKey)
+            {
+                _id = id;
+                _affKey = affKey;
+            }
+
+            /** <inheritdoc /> */
+            public override bool Equals(object obj)
+            {
+                AffinityTestKey other = obj as AffinityTestKey;
+
+                return other != null && _id == other._id;
+            }
+
+            /** <inheritdoc /> */
+            public override int GetHashCode()
+            {
+                return _id;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheDynamicStartTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheDynamicStartTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheDynamicStartTest.cs
new file mode 100644
index 0000000..210d80c
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheDynamicStartTest.cs
@@ -0,0 +1,281 @@
+/*
+ * 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.Cache
+{
+    using System;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.Portable;
+    using Apache.Ignite.Core.Tests.Query;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests for dynamic a cache start.
+    /// </summary>
+    public class CacheDynamicStartTest
+    {
+        /** Grid name: data. */
+        private const string GridData = "d";
+
+        /** Grid name: data, no configuration. */
+        private const string GridDataNoCfg = "dnc";
+
+        /** Grid name: client. */
+        private const string GridClient = "c";
+
+        /** Cache name: partitioned, transactional. */
+        private const string CacheTx = "p";
+
+        /** Cache name: atomic. */
+        private const string CacheAtomic = "pa";
+
+        /** Cache name: dummy. */
+        private const string CacheDummy = "dummy";
+        
+        /// <summary>
+        /// Set up routine.
+        /// </summary>
+        [SetUp]
+        public void SetUp()
+        {
+            TestUtils.KillProcesses();
+
+            Ignition.Start(CreateConfiguration(GridData, @"config/dynamic/dynamic-data.xml"));
+            Ignition.Start(CreateConfiguration(GridDataNoCfg, @"config/dynamic/dynamic-data-no-cfg.xml"));
+            Ignition.Start(CreateConfiguration(GridClient, @"config/dynamic/dynamic-client.xml"));
+        }
+
+        /// <summary>
+        /// Tear down routine.
+        /// </summary>
+        [TearDown]
+        public void StopGrids()
+        {
+            Ignition.Stop(GridData, true);
+            Ignition.Stop(GridDataNoCfg, true);
+            Ignition.Stop(GridClient, true);
+        }
+
+        /// <summary>
+        /// Create configuration.
+        /// </summary>
+        /// <param name="name">Grid name.</param>
+        /// <param name="springCfg">Spring configuration.</param>
+        /// <returns>Configuration.</returns>
+        private static IgniteConfigurationEx CreateConfiguration(string name, string springCfg)
+        {
+            IgniteConfigurationEx cfg = new IgniteConfigurationEx();
+
+            PortableConfiguration portCfg = new PortableConfiguration();
+
+            ICollection<PortableTypeConfiguration> portTypeCfgs = new List<PortableTypeConfiguration>();
+
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(DynamicTestKey)));
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(DynamicTestValue)));
+
+            portCfg.TypeConfigurations = portTypeCfgs;
+
+            cfg.GridName = name;
+            cfg.PortableConfiguration = portCfg;
+            cfg.JvmClasspath = TestUtils.CreateTestClasspath();
+            cfg.JvmOptions = TestUtils.TestJavaOptions();
+            cfg.SpringConfigUrl = springCfg;
+
+            return cfg;
+        }
+
+        /// <summary>
+        /// Try getting not configured cache.
+        /// </summary>
+        [Test]
+        public void TestNoStarted()
+        {
+            Assert.Throws<ArgumentException>(() =>
+            {
+                Ignition.GetIgnite(GridData).Cache<CacheTestKey, PortablePerson>(CacheDummy);
+            });
+
+            Assert.Throws<ArgumentException>(() =>
+            {
+                Ignition.GetIgnite(GridDataNoCfg).Cache<CacheTestKey, PortablePerson>(CacheDummy);
+            });
+
+            Assert.Throws<ArgumentException>(() =>
+            {
+                Ignition.GetIgnite(GridClient).Cache<CacheTestKey, PortablePerson>(CacheDummy);
+            });
+        }
+
+        /// <summary>
+        /// Test TX cache.
+        /// </summary>
+        [Test]
+        public void TestTransactional()
+        {
+            Check(CacheTx);
+        }
+
+        /// <summary>
+        /// Test ATOMIC cache.
+        /// </summary>
+        [Test]
+        public void TestAtomic()
+        {
+            Check(CacheAtomic);
+        }
+
+        /// <summary>
+        /// Check routine.
+        /// </summary>
+        /// <param name="cacheName">Cache name.</param>
+        private void Check(string cacheName)
+        {
+            ICache<DynamicTestKey, DynamicTestValue> cacheData =
+                Ignition.GetIgnite(GridData).Cache<DynamicTestKey, DynamicTestValue>(cacheName);
+
+            ICache<DynamicTestKey, DynamicTestValue> cacheDataNoCfg =
+                Ignition.GetIgnite(GridDataNoCfg).Cache<DynamicTestKey, DynamicTestValue>(cacheName);
+
+            ICache<DynamicTestKey, DynamicTestValue> cacheClient =
+                Ignition.GetIgnite(GridClient).Cache<DynamicTestKey, DynamicTestValue>(cacheName);
+
+            DynamicTestKey key1 = new DynamicTestKey(1);
+            DynamicTestKey key2 = new DynamicTestKey(2);
+            DynamicTestKey key3 = new DynamicTestKey(3);
+
+            DynamicTestValue val1 = new DynamicTestValue(1);
+            DynamicTestValue val2 = new DynamicTestValue(2);
+            DynamicTestValue val3 = new DynamicTestValue(3);
+
+            cacheData.Put(key1, val1);
+            Assert.AreEqual(val1, cacheData.Get(key1));
+            Assert.AreEqual(val1, cacheDataNoCfg.Get(key1));
+            Assert.AreEqual(val1, cacheClient.Get(key1));
+
+            cacheDataNoCfg.Put(key2, val2);
+            Assert.AreEqual(val2, cacheData.Get(key2));
+            Assert.AreEqual(val2, cacheDataNoCfg.Get(key2));
+            Assert.AreEqual(val2, cacheClient.Get(key2));
+
+            cacheClient.Put(key3, val3);
+            Assert.AreEqual(val3, cacheData.Get(key3));
+            Assert.AreEqual(val3, cacheDataNoCfg.Get(key3));
+            Assert.AreEqual(val3, cacheClient.Get(key3));
+
+            for (int i = 0; i < 10000; i++)
+                cacheClient.Put(new DynamicTestKey(i), new DynamicTestValue(1));
+
+            int sizeClient = cacheClient.LocalSize();
+
+            Assert.AreEqual(0, sizeClient);
+        }
+    }
+
+    /// <summary>
+    /// Key for dynamic cache start tests.
+    /// </summary>
+    class DynamicTestKey
+    {
+        /// <summary>
+        /// Default constructor.
+        /// </summary>
+        public DynamicTestKey()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="id">ID.</param>
+        public DynamicTestKey(int id)
+        {
+            Id = id;
+        }
+
+        /// <summary>
+        /// ID.
+        /// </summary>
+        public int Id
+        {
+            get;
+            set;
+        }
+
+        /** <inheritdoc /> */
+        public override bool Equals(object obj)
+        {
+            DynamicTestKey other = obj as DynamicTestKey;
+
+            return other != null && Id == other.Id;
+        }
+
+        /** <inheritdoc /> */
+        public override int GetHashCode()
+        {
+            return Id;
+        }
+    }
+
+    /// <summary>
+    /// Value for dynamic cache start tests.
+    /// </summary>
+    class DynamicTestValue
+    {
+        /// <summary>
+        /// Default constructor.
+        /// </summary>
+        public DynamicTestValue()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="id">ID.</param>
+        public DynamicTestValue(int id)
+        {
+            Id = id;
+        }
+
+        /// <summary>
+        /// ID.
+        /// </summary>
+        public int Id
+        {
+            get;
+            set;
+        }
+
+        /** <inheritdoc /> */
+        public override bool Equals(object obj)
+        {
+            DynamicTestValue other = obj as DynamicTestValue;
+
+            return other != null && Id == other.Id;
+        }
+
+        /** <inheritdoc /> */
+        public override int GetHashCode()
+        {
+            return Id;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheEntryTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheEntryTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheEntryTest.cs
new file mode 100644
index 0000000..8464b8e
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheEntryTest.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.Cache
+{
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Impl.Cache;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// <see cref="CacheEntry{TK,TV}"/> tests.
+    /// </summary>
+    public class CacheEntryTest
+    {
+        /// <summary>
+        /// Tests equality members.
+        /// </summary>
+        [Test]
+        public void TestEquality()
+        {
+            var entry1 = new CacheEntry<int, int>(1, 2);
+            var entry2 = new CacheEntry<int, int>(1, 2);
+            var entry3 = new CacheEntry<int, int>(1, 3);
+
+            Assert.AreEqual(entry1, entry2);
+            Assert.AreNotEqual(entry1, entry3);
+
+            var boxedEntry1 = (object) entry1;
+            var boxedEntry2 = (object) entry2;
+            var boxedEntry3 = (object) entry3;
+
+            Assert.IsFalse(ReferenceEquals(boxedEntry1, boxedEntry2));
+
+            Assert.AreEqual(boxedEntry1, boxedEntry2);
+            Assert.AreNotEqual(boxedEntry1, boxedEntry3);
+        }
+
+        /// <summary>
+        /// Tests with hash data structures.
+        /// </summary>
+        [Test]
+        public void TestHashCode()
+        {
+            var entry1 = new CacheEntry<int, int>(1, 2);
+            var entry2 = new CacheEntry<int, int>(1, 2);
+            var entry3 = new CacheEntry<int, int>(1, 3);
+
+            var set = new HashSet<object> {entry1};
+
+            Assert.IsTrue(set.Contains(entry1));
+            Assert.IsTrue(set.Contains(entry2));
+            Assert.IsFalse(set.Contains(entry3));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheForkedTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheForkedTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheForkedTest.cs
new file mode 100644
index 0000000..04aff5f
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheForkedTest.cs
@@ -0,0 +1,82 @@
+/*
+ * 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.Cache
+{
+    using System.IO;
+    using Apache.Ignite.Core.Tests.Process;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests cache with a standalone process.
+    /// </summary>
+    [Ignore("IGNITE-1367")]
+    public class CacheForkedTest
+    {
+        /** */
+        private IIgnite _grid;
+
+        /// <summary>
+        /// Set up.
+        /// </summary>
+        [TestFixtureSetUp]
+        public void SetUp()
+        {
+            const string springConfigUrl = "config\\compute\\compute-grid1.xml";
+            
+            // ReSharper disable once UnusedVariable
+            var proc = new IgniteProcess(
+                "-jvmClasspath=" + TestUtils.CreateTestClasspath(),
+                "-springConfigUrl=" + Path.GetFullPath(springConfigUrl),
+                "-J-ea",
+                "-J-Xcheck:jni",
+                "-J-Xms512m",
+                "-J-Xmx512m",
+                "-J-DIGNITE_QUIET=false"
+                );
+
+            _grid = Ignition.Start(new IgniteConfiguration
+            {
+                JvmClasspath = TestUtils.CreateTestClasspath(),
+                JvmOptions = TestUtils.TestJavaOptions(),
+                SpringConfigUrl = springConfigUrl
+            });
+
+            Assert.IsTrue(_grid.WaitTopology(2, 30000));
+        }
+
+        /// <summary>
+        /// Tear down.
+        /// </summary>
+        [TestFixtureTearDown]
+        public void TearDown()
+        {
+            IgniteProcess.KillAll();
+
+            Ignition.StopAll(true);
+        }
+
+        /// <summary>
+        /// Tests cache clear.
+        /// </summary>
+        [Test]
+        public void TestClearCache()
+        {
+            _grid.Cache<object, object>(null).Clear();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheLocalAtomicTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheLocalAtomicTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheLocalAtomicTest.cs
new file mode 100644
index 0000000..b60c254
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheLocalAtomicTest.cs
@@ -0,0 +1,57 @@
+/*
+ * 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.Cache
+{
+    public class CacheLocalAtomicTest : CacheAbstractTest
+    {
+        protected override int CachePartitions()
+        {
+            return 1;
+        }
+
+        protected override int GridCount()
+        {
+            return 1;
+        }
+
+        protected override string CacheName()
+        {
+            return "local_atomic";
+        }
+
+        protected override bool NearEnabled()
+        {
+            return false;
+        }
+
+        protected override bool TxEnabled()
+        {
+            return false;
+        }
+
+        protected override bool LocalCache()
+        {
+            return true;
+        }
+
+        protected override int Backups()
+        {
+            return 0;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheLocalTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheLocalTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheLocalTest.cs
new file mode 100644
index 0000000..02cb987
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheLocalTest.cs
@@ -0,0 +1,56 @@
+/*
+ * 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.Cache
+{
+    public class CacheLocalTest : CacheAbstractTest
+    {
+        protected override int CachePartitions()
+        {
+            return 1;
+        }
+
+        protected override int GridCount()
+        {
+            return 1;
+        }
+
+        protected override string CacheName()
+        {
+            return "local";
+        }
+
+        protected override bool NearEnabled()
+        {
+            return false;
+        }
+
+        protected override bool TxEnabled()
+        {
+            return true;
+        }
+        protected override bool LocalCache()
+        {
+            return true;
+        }
+
+        protected override int Backups()
+        {
+            return 0;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedAtomicNearEnabledTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedAtomicNearEnabledTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedAtomicNearEnabledTest.cs
new file mode 100644
index 0000000..4f6e7a0
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedAtomicNearEnabledTest.cs
@@ -0,0 +1,50 @@
+/*
+ * 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.Cache
+{
+    using NUnit.Framework;
+
+    [Category(TestUtils.CategoryIntensive)]
+    public class CachePartitionedAtomicNearEnabledTest : CacheAbstractTest
+    {
+        protected override int GridCount()
+        {
+            return 3;
+        }
+
+        protected override string CacheName()
+        {
+            return "partitioned_atomic_near";
+        }
+
+        protected override bool NearEnabled()
+        {
+            return true;
+        }
+
+        protected override bool TxEnabled()
+        {
+            return false;
+        }
+
+        protected override int Backups()
+        {
+            return 1;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedAtomicTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedAtomicTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedAtomicTest.cs
new file mode 100644
index 0000000..ab59c64
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedAtomicTest.cs
@@ -0,0 +1,50 @@
+/*
+ * 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.Cache
+{
+    using NUnit.Framework;
+
+    [Category(TestUtils.CategoryIntensive)]
+    public class CachePartitionedAtomicTest : CacheAbstractTest
+    {
+        protected override int GridCount()
+        {
+            return 3;
+        }
+
+        protected override string CacheName()
+        {
+            return "partitioned_atomic";
+        }
+
+        protected override bool NearEnabled()
+        {
+            return false;
+        }
+
+        protected override bool TxEnabled()
+        {
+            return false;
+        }
+
+        protected override int Backups()
+        {
+            return 1;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedNearEnabledTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedNearEnabledTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedNearEnabledTest.cs
new file mode 100644
index 0000000..830698b
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedNearEnabledTest.cs
@@ -0,0 +1,50 @@
+/*
+ * 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.Cache
+{
+    using NUnit.Framework;
+
+    [Category(TestUtils.CategoryIntensive)]
+    public class CachePartitionedNearEnabledTest : CacheAbstractTest
+    {
+        protected override int GridCount()
+        {
+            return 3;
+        }
+
+        protected override string CacheName()
+        {
+            return "partitioned_near";
+        }
+
+        protected override bool NearEnabled()
+        {
+            return true;
+        }
+
+        protected override bool TxEnabled()
+        {
+            return true;
+        }
+
+        protected override int Backups()
+        {
+            return 1;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedTest.cs
new file mode 100644
index 0000000..02d3208
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedTest.cs
@@ -0,0 +1,50 @@
+/*
+ * 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.Cache
+{
+    using NUnit.Framework;
+
+    [Category(TestUtils.CategoryIntensive)]
+    public class CachePartitionedTest : CacheAbstractTest
+    {
+        protected override int GridCount()
+        {
+            return 3;
+        }
+
+        protected override string CacheName()
+        {
+            return "partitioned";
+        }
+
+        protected override bool NearEnabled()
+        {
+            return false;
+        }
+
+        protected override bool TxEnabled()
+        {
+            return true;
+        }
+
+        protected override int Backups()
+        {
+            return 1;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheReplicatedAtomicTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheReplicatedAtomicTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheReplicatedAtomicTest.cs
new file mode 100644
index 0000000..db6f5a5
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheReplicatedAtomicTest.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.Tests.Cache
+{
+    using NUnit.Framework;
+
+    [Category(TestUtils.CategoryIntensive)]
+    public class CacheReplicatedAtomicTest : CacheAbstractTest
+    {
+        protected override int CachePartitions()
+        {
+            return 512;
+        }
+
+        protected override int GridCount()
+        {
+            return 3;
+        }
+
+        protected override string CacheName()
+        {
+            return "replicated_atomic";
+        }
+
+        protected override bool NearEnabled()
+        {
+            return false;
+        }
+
+        protected override bool TxEnabled()
+        {
+            return false;
+        }
+
+        protected override int Backups()
+        {
+            return GridCount() - 1;
+        }
+
+        protected override bool ReplicatedCache()
+        {
+            return true;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheReplicatedTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheReplicatedTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheReplicatedTest.cs
new file mode 100644
index 0000000..7c70222
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheReplicatedTest.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.Tests.Cache
+{
+    using NUnit.Framework;
+
+    [Category(TestUtils.CategoryIntensive)]
+    public class CacheReplicatedTest : CacheAbstractTest
+    {
+        protected override int CachePartitions()
+        {
+            return 512;
+        }
+
+        protected override int GridCount()
+        {
+            return 3;
+        }
+
+        protected override string CacheName()
+        {
+            return "replicated";
+        }
+
+        protected override bool NearEnabled()
+        {
+            return false;
+        }
+
+        protected override bool TxEnabled()
+        {
+            return true;
+        }
+
+        protected override int Backups()
+        {
+            return GridCount() - 1;
+        }
+
+        protected override bool ReplicatedCache()
+        {
+            return true;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheTestAsyncWrapper.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheTestAsyncWrapper.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheTestAsyncWrapper.cs
new file mode 100644
index 0000000..93f5973
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheTestAsyncWrapper.cs
@@ -0,0 +1,436 @@
+/*
+ * 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.Cache
+{
+    using System.Collections;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cache.Expiry;
+    using Apache.Ignite.Core.Cache.Query;
+    using Apache.Ignite.Core.Cache.Query.Continuous;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// Wraps IGridCache implementation to simplify async mode testing.
+    /// </summary>
+    internal class CacheTestAsyncWrapper<TK, TV> : ICache<TK, TV>
+    {
+        private readonly ICache<TK, TV> _cache;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheTestAsyncWrapper{K, V}"/> class.
+        /// </summary>
+        /// <param name="cache">The cache to be wrapped.</param>
+        public CacheTestAsyncWrapper(ICache<TK, TV> cache)
+        {
+            Debug.Assert(cache.IsAsync, "GridCacheTestAsyncWrapper only works with async caches.");
+
+            _cache = cache;
+        }
+
+        /** <inheritDoc /> */
+        public ICache<TK, TV> WithAsync()
+        {
+            return this;
+        }
+
+        /** <inheritDoc /> */
+        public bool IsAsync
+        {
+            get { return true; }
+        }
+
+        /** <inheritDoc /> */
+        public IFuture GetFuture()
+        {
+            Debug.Fail("GridCacheTestAsyncWrapper.Future() should not be called. It always returns null.");
+            return null;
+        }
+
+        /** <inheritDoc /> */
+        public IFuture<TResult> GetFuture<TResult>()
+        {
+            Debug.Fail("GridCacheTestAsyncWrapper.Future() should not be called. It always returns null.");
+            return null;
+        }
+
+        /** <inheritDoc /> */
+        public string Name
+        {
+            get { return _cache.Name; }
+        }
+
+        /** <inheritDoc /> */
+        public IIgnite Ignite
+        {
+            get { return _cache.Ignite; }
+        }
+
+        /** <inheritDoc /> */
+        public bool IsEmpty
+        {
+            get { return _cache.IsEmpty; }
+        }
+
+        /** <inheritDoc /> */
+        public bool KeepPortable
+        {
+            get { return _cache.KeepPortable; }
+        }
+
+        /** <inheritDoc /> */
+        public ICache<TK, TV> WithSkipStore()
+        {
+            return _cache.WithSkipStore().WrapAsync();
+        }
+
+        /** <inheritDoc /> */
+        public ICache<TK, TV> WithExpiryPolicy(IExpiryPolicy plc)
+        {
+            return _cache.WithExpiryPolicy(plc).WrapAsync();
+        }
+
+        /** <inheritDoc /> */
+        public ICache<TK1, TV1> WithKeepPortable<TK1, TV1>()
+        {
+            return _cache.WithKeepPortable<TK1, TV1>().WrapAsync();
+        }
+        
+        /** <inheritDoc /> */
+        public void LoadCache(ICacheEntryFilter<TK, TV> p, params object[] args)
+        {
+            _cache.LoadCache(p, args);
+            WaitResult();
+        }
+
+        /** <inheritDoc /> */
+        public void LocalLoadCache(ICacheEntryFilter<TK, TV> p, params object[] args)
+        {
+            _cache.LocalLoadCache(p, args);
+            WaitResult();
+        }
+
+        /** <inheritDoc /> */
+        public bool ContainsKey(TK key)
+        {
+            _cache.ContainsKey(key);
+            return GetResult<bool>();
+        }
+
+        /** <inheritDoc /> */
+        public bool ContainsKeys(IEnumerable<TK> keys)
+        {
+            _cache.ContainsKeys(keys);
+            return GetResult<bool>();
+        }
+
+        /** <inheritDoc /> */
+        public TV LocalPeek(TK key, params CachePeekMode[] modes)
+        {
+            _cache.LocalPeek(key, modes);
+            return GetResult<TV>();
+        }
+
+        /** <inheritDoc /> */
+        public TV Get(TK key)
+        {
+            _cache.Get(key);
+            return GetResult<TV>();
+        }
+
+        /** <inheritDoc /> */
+        public IDictionary<TK, TV> GetAll(IEnumerable<TK> keys)
+        {
+            _cache.GetAll(keys);
+            return GetResult<IDictionary<TK, TV>>();
+        }
+
+        /** <inheritDoc /> */
+        public void Put(TK key, TV val)
+        {
+            _cache.Put(key, val);
+            WaitResult();
+        }
+
+        /** <inheritDoc /> */
+        public TV GetAndPut(TK key, TV val)
+        {
+            _cache.GetAndPut(key, val);
+            return GetResult<TV>();
+        }
+
+        /** <inheritDoc /> */
+        public TV GetAndReplace(TK key, TV val)
+        {
+            _cache.GetAndReplace(key, val);
+            return GetResult<TV>();
+        }
+
+        /** <inheritDoc /> */
+        public TV GetAndRemove(TK key)
+        {
+            _cache.GetAndRemove(key);
+            return GetResult<TV>();
+        }
+
+        /** <inheritDoc /> */
+        public bool PutIfAbsent(TK key, TV val)
+        {
+            _cache.PutIfAbsent(key, val);
+            return GetResult<bool>();
+        }
+
+        /** <inheritDoc /> */
+        public TV GetAndPutIfAbsent(TK key, TV val)
+        {
+            _cache.GetAndPutIfAbsent(key, val);
+            return GetResult<TV>();
+        }
+
+        /** <inheritDoc /> */
+        public bool Replace(TK key, TV val)
+        {
+            _cache.Replace(key, val);
+            return GetResult<bool>();
+        }
+
+        /** <inheritDoc /> */
+        public bool Replace(TK key, TV oldVal, TV newVal)
+        {
+            _cache.Replace(key, oldVal, newVal);
+            return GetResult<bool>();
+        }
+
+        /** <inheritDoc /> */
+        public void PutAll(IDictionary<TK, TV> vals)
+        {
+            _cache.PutAll(vals);
+            WaitResult();
+        }
+
+        /** <inheritDoc /> */
+        public void LocalEvict(IEnumerable<TK> keys)
+        {
+            _cache.LocalEvict(keys);
+        }
+
+        /** <inheritDoc /> */
+        public void Clear()
+        {
+            _cache.Clear();
+            WaitResult();
+        }
+
+        /** <inheritDoc /> */
+        public void Clear(TK key)
+        {
+            _cache.Clear(key);
+        }
+
+        /** <inheritDoc /> */
+        public void ClearAll(IEnumerable<TK> keys)
+        {
+            _cache.ClearAll(keys);
+        }
+
+        /** <inheritDoc /> */
+        public void LocalClear(TK key)
+        {
+            _cache.LocalClear(key);
+        }
+
+        /** <inheritDoc /> */
+        public void LocalClearAll(IEnumerable<TK> keys)
+        {
+            _cache.LocalClearAll(keys);
+        }
+
+        /** <inheritDoc /> */
+        public bool Remove(TK key)
+        {
+            _cache.Remove(key);
+            return GetResult<bool>();
+        }
+
+        /** <inheritDoc /> */
+        public bool Remove(TK key, TV val)
+        {
+            _cache.Remove(key, val);
+            return GetResult<bool>();
+        }
+
+        /** <inheritDoc /> */
+        public void RemoveAll(IEnumerable<TK> keys)
+        {
+            _cache.RemoveAll(keys);
+            WaitResult();
+        }
+
+        /** <inheritDoc /> */
+        public void RemoveAll()
+        {
+            _cache.RemoveAll();
+            WaitResult();
+        }
+
+        /** <inheritDoc /> */
+        public int LocalSize(params CachePeekMode[] modes)
+        {
+            return _cache.LocalSize(modes);
+        }
+
+        /** <inheritDoc /> */
+        public int Size(params CachePeekMode[] modes)
+        {
+            _cache.Size(modes);
+            return GetResult<int>();
+        }
+
+        /** <inheritDoc /> */
+        public void LocalPromote(IEnumerable<TK> keys)
+        {
+            _cache.LocalPromote(keys);
+        }
+        
+        /** <inheritDoc /> */
+        public IQueryCursor<ICacheEntry<TK, TV>> Query(QueryBase qry)
+        {
+            return _cache.Query(qry);
+        }
+
+        /** <inheritDoc /> */
+        public IQueryCursor<IList> QueryFields(SqlFieldsQuery qry)
+        {
+            return _cache.QueryFields(qry);
+        }
+
+        /** <inheritDoc /> */
+        IContinuousQueryHandle ICache<TK, TV>.QueryContinuous(ContinuousQuery<TK, TV> qry)
+        {
+            return _cache.QueryContinuous(qry);
+        }
+
+        /** <inheritDoc /> */
+        public IContinuousQueryHandle<ICacheEntry<TK, TV>> QueryContinuous(ContinuousQuery<TK, TV> qry, QueryBase initialQry)
+        {
+            return _cache.QueryContinuous(qry, initialQry);
+        }
+
+        /** <inheritDoc /> */
+        public IEnumerable<ICacheEntry<TK, TV>> GetLocalEntries(params CachePeekMode[] peekModes)
+        {
+            return _cache.GetLocalEntries(peekModes);
+        }
+
+        /** <inheritDoc /> */
+        public TR Invoke<TR, TA>(TK key, ICacheEntryProcessor<TK, TV, TA, TR> processor, TA arg)
+        {
+            _cache.Invoke(key, processor, arg);
+            
+            return GetResult<TR>();
+        }
+
+        /** <inheritDoc /> */
+        public IDictionary<TK, ICacheEntryProcessorResult<TR>> InvokeAll<TR, TA>(IEnumerable<TK> keys, 
+            ICacheEntryProcessor<TK, TV, TA, TR> processor, TA arg)
+        {
+            _cache.InvokeAll(keys, processor, arg);
+
+            return GetResult<IDictionary<TK, ICacheEntryProcessorResult<TR>>>();
+        }
+
+        /** <inheritDoc /> */
+        public ICacheLock Lock(TK key)
+        {
+            return _cache.Lock(key);
+        }
+
+        /** <inheritDoc /> */
+        public ICacheLock LockAll(IEnumerable<TK> keys)
+        {
+            return _cache.LockAll(keys);
+        }
+
+        /** <inheritDoc /> */
+        public bool IsLocalLocked(TK key, bool byCurrentThread)
+        {
+            return _cache.IsLocalLocked(key, byCurrentThread);
+        }
+
+        /** <inheritDoc /> */
+        public ICacheMetrics GetMetrics()
+        {
+            return _cache.GetMetrics();
+        }
+
+        /** <inheritDoc /> */
+        public IFuture Rebalance()
+        {
+            return _cache.Rebalance();
+        }
+
+        /** <inheritDoc /> */
+        public ICache<TK, TV> WithNoRetries()
+        {
+            return _cache.WithNoRetries();
+        }
+
+        /** <inheritDoc /> */
+        public IEnumerator<ICacheEntry<TK, TV>> GetEnumerator()
+        {
+            return _cache.GetEnumerator();
+        }
+
+        /** <inheritDoc /> */
+        IEnumerator IEnumerable.GetEnumerator()
+        {
+            return GetEnumerator();
+        }
+
+        /// <summary>
+        /// Waits for the async result.
+        /// </summary>
+        private void WaitResult()
+        {
+            GetResult<object>();
+        }
+
+        /// <summary>
+        /// Gets the async result.
+        /// </summary>
+        private T GetResult<T>()
+        {
+            return _cache.GetFuture<T>().Get();
+        }
+    }
+
+    /// <summary>
+    /// Extension methods for IGridCache.
+    /// </summary>
+    public static class CacheExtensions
+    {
+        /// <summary>
+        /// Wraps specified instance into GridCacheTestAsyncWrapper.
+        /// </summary>
+        public static ICache<TK, TV> WrapAsync<TK, TV>(this ICache<TK, TV> cache)
+        {
+            return new CacheTestAsyncWrapper<TK, TV>(cache);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
new file mode 100644
index 0000000..85227b6
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
@@ -0,0 +1,928 @@
+/*
+ * 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.Cache.Query
+{
+    using System;
+    using System.Collections;
+    using System.Collections.Generic;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Text;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cache.Query;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Portable;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Queries tests.
+    /// </summary>
+    public class CacheQueriesTest
+    {
+        /** Grid count. */
+        private const int GridCnt = 2;
+
+        /** Cache name. */
+        private const string CacheName = "cache";
+
+        /** Path to XML configuration. */
+        private const string CfgPath = "config\\cache-query.xml";
+
+        /** Maximum amount of items in cache. */
+        private const int MaxItemCnt = 100;
+
+        /// <summary>
+        /// 
+        /// </summary>
+        [TestFixtureSetUp]
+        public virtual void StartGrids()
+        {
+            TestUtils.JvmDebug = true;
+            TestUtils.KillProcesses();
+
+            IgniteConfigurationEx cfg = new IgniteConfigurationEx
+            {
+                PortableConfiguration = new PortableConfiguration
+                {
+                    TypeConfigurations = new[]
+                    {
+                        new PortableTypeConfiguration(typeof (QueryPerson)),
+                        new PortableTypeConfiguration(typeof (PortableScanQueryFilter<QueryPerson>)),
+                        new PortableTypeConfiguration(typeof (PortableScanQueryFilter<PortableUserObject>))
+                    }
+                },
+                JvmClasspath = TestUtils.CreateTestClasspath(),
+                JvmOptions = TestUtils.TestJavaOptions(),
+                SpringConfigUrl = CfgPath
+            };
+
+            for (int i = 0; i < GridCnt; i++)
+            {
+                cfg.GridName = "grid-" + i;
+
+                Ignition.Start(cfg);
+            }
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        [TestFixtureTearDown]
+        public virtual void StopGrids()
+        {
+            for (int i = 0; i < GridCnt; i++)
+                Ignition.Stop("grid-" + i, true);
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        [SetUp]
+        public virtual void BeforeTest()
+        {
+            Console.WriteLine("Test started: " + TestContext.CurrentContext.Test.Name);
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        [TearDown]
+        public virtual void AfterTest()
+        {
+            var cache = Cache();
+
+            for (int i = 0; i < GridCnt; i++)
+            {
+                for (int j = 0; j < MaxItemCnt; j++)
+                    cache.Remove(j);
+
+                Assert.IsTrue(cache.IsEmpty);
+            }
+
+            Console.WriteLine("Test finished: " + TestContext.CurrentContext.Test.Name);
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        /// <param name="idx"></param>
+        /// <returns></returns>
+        public IIgnite GetIgnite(int idx)
+        {
+            return Ignition.GetIgnite("grid-" + idx);
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        /// <param name="idx"></param>
+        /// <returns></returns>
+        public ICache<int, QueryPerson> Cache(int idx)
+        {
+            return GetIgnite(idx).Cache<int, QueryPerson>(CacheName);
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        /// <returns></returns>
+        public ICache<int, QueryPerson> Cache()
+        {
+            return Cache(0);
+        }
+
+        /// <summary>
+        /// Test arguments validation for SQL queries.
+        /// </summary>
+        [Test]
+        public void TestValidationSql()
+        {
+            // 1. No sql.
+            Assert.Throws<ArgumentException>(() =>
+                { Cache().Query(new SqlQuery(typeof(QueryPerson), null)); });
+
+            // 2. No type.
+            Assert.Throws<ArgumentException>(() =>
+                { Cache().Query(new SqlQuery((string)null, "age >= 50")); });
+        }
+
+        /// <summary>
+        /// Test arguments validation for SQL fields queries.
+        /// </summary>
+        [Test]
+        public void TestValidationSqlFields()
+        {
+            // 1. No sql.
+            Assert.Throws<ArgumentException>(() => { Cache().QueryFields(new SqlFieldsQuery(null)); });
+        }
+
+        /// <summary>
+        /// Test arguments validation for TEXT queries.
+        /// </summary>
+        [Test]
+        public void TestValidationText()
+        {
+            // 1. No text.
+            Assert.Throws<ArgumentException>(() =>
+                { Cache().Query(new TextQuery(typeof(QueryPerson), null)); });
+
+            // 2. No type.
+            Assert.Throws<ArgumentException>(() =>
+                { Cache().Query(new TextQuery((string)null, "Ivanov")); });
+        }
+
+        /// <summary>
+        /// Cursor tests.
+        /// </summary>
+        [Test]
+        [SuppressMessage("ReSharper", "ReturnValueOfPureMethodIsNotUsed")]
+        public void TestCursor()
+        {
+            var cache0 = Cache().WithAsync();
+
+            cache0.WithAsync().Put(1, new QueryPerson("Ivanov", 30));
+
+            IFuture<object> res = cache0.GetFuture<object>();
+
+            res.Get();
+
+            Cache().Put(1, new QueryPerson("Ivanov", 30));
+            Cache().Put(1, new QueryPerson("Petrov", 40));
+            Cache().Put(1, new QueryPerson("Sidorov", 50));
+
+            SqlQuery qry = new SqlQuery(typeof(QueryPerson), "age >= 20");
+
+            // 1. Test GetAll().
+            using (IQueryCursor<ICacheEntry<int, QueryPerson>> cursor = Cache().Query(qry))
+            {
+                cursor.GetAll();
+
+                Assert.Throws<InvalidOperationException>(() => { cursor.GetAll(); });
+                Assert.Throws<InvalidOperationException>(() => { cursor.GetEnumerator(); });
+            }
+
+            // 2. Test GetEnumerator.
+            using (IQueryCursor<ICacheEntry<int, QueryPerson>> cursor = Cache().Query(qry))
+            {
+                cursor.GetEnumerator();
+
+                Assert.Throws<InvalidOperationException>(() => { cursor.GetAll(); });
+                Assert.Throws<InvalidOperationException>(() => { cursor.GetEnumerator(); });
+            }
+        }
+
+        /// <summary>
+        /// Test enumerator.
+        /// </summary>
+        [Test]
+        [SuppressMessage("ReSharper", "UnusedVariable")]
+        public void TestEnumerator()
+        {
+            Cache().Put(1, new QueryPerson("Ivanov", 30));
+            Cache().Put(2, new QueryPerson("Petrov", 40));
+            Cache().Put(3, new QueryPerson("Sidorov", 50));
+            Cache().Put(4, new QueryPerson("Unknown", 60));
+
+            // 1. Empty result set.
+            using (
+                IQueryCursor<ICacheEntry<int, QueryPerson>> cursor =
+                    Cache().Query(new SqlQuery(typeof(QueryPerson), "age = 100")))
+            {
+                IEnumerator<ICacheEntry<int, QueryPerson>> e = cursor.GetEnumerator();
+
+                Assert.Throws<InvalidOperationException>(() =>
+                    { ICacheEntry<int, QueryPerson> entry = e.Current; });
+
+                Assert.IsFalse(e.MoveNext());
+
+                Assert.Throws<InvalidOperationException>(() =>
+                    { ICacheEntry<int, QueryPerson> entry = e.Current; });
+
+                Assert.Throws<NotSupportedException>(() => e.Reset());
+            }
+
+            SqlQuery qry = new SqlQuery(typeof (QueryPerson), "age < 60");
+
+            // 2. Page size is bigger than result set.
+            qry.PageSize = 4;
+            CheckEnumeratorQuery(qry);
+
+            // 3. Page size equal to result set.
+            qry.PageSize = 3;
+            CheckEnumeratorQuery(qry);
+
+            // 4. Page size if less than result set.
+            qry.PageSize = 2;
+            CheckEnumeratorQuery(qry);
+        }
+
+        /// <summary>
+        /// Test SQL query arguments passing.
+        /// </summary>
+        public void TestSqlQueryArguments()
+        {
+            Cache().Put(1, new QueryPerson("Ivanov", 30));
+            Cache().Put(2, new QueryPerson("Petrov", 40));
+            Cache().Put(3, new QueryPerson("Sidorov", 50));
+
+            // 1. Empty result set.
+            using (
+                IQueryCursor<ICacheEntry<int, QueryPerson>> cursor =
+                    Cache().Query(new SqlQuery(typeof(QueryPerson), "age < ?", 50)))
+            {
+                foreach (ICacheEntry<int, QueryPerson> entry in cursor.GetAll())
+                    Assert.IsTrue(entry.Key == 1 || entry.Key == 2);
+            }
+        }
+
+        /// <summary>
+        /// Test SQL fields query arguments passing.
+        /// </summary>
+        public void TestSqlFieldsQueryArguments()
+        {
+            Cache().Put(1, new QueryPerson("Ivanov", 30));
+            Cache().Put(2, new QueryPerson("Petrov", 40));
+            Cache().Put(3, new QueryPerson("Sidorov", 50));
+
+            // 1. Empty result set.
+            using (
+                IQueryCursor<IList> cursor = Cache().QueryFields(
+                    new SqlFieldsQuery("SELECT age FROM QueryPerson WHERE age < ?", 50)))
+            {
+                foreach (IList entry in cursor.GetAll())
+                    Assert.IsTrue((int) entry[0] < 50);
+            }
+        }
+
+        /// <summary>
+        /// Check query result for enumerator test.
+        /// </summary>
+        /// <param name="qry">QUery.</param>
+        private void CheckEnumeratorQuery(SqlQuery qry)
+        {
+            using (IQueryCursor<ICacheEntry<int, QueryPerson>> cursor = Cache().Query(qry))
+            {
+                bool first = false;
+                bool second = false;
+                bool third = false;
+
+                foreach (var entry in cursor)
+                {
+                    if (entry.Key == 1)
+                    {
+                        first = true;
+
+                        Assert.AreEqual("Ivanov", entry.Value.Name);
+                        Assert.AreEqual(30, entry.Value.Age);
+                    }
+                    else if (entry.Key == 2)
+                    {
+                        second = true;
+
+                        Assert.AreEqual("Petrov", entry.Value.Name);
+                        Assert.AreEqual(40, entry.Value.Age);
+                    }
+                    else if (entry.Key == 3)
+                    {
+                        third = true;
+
+                        Assert.AreEqual("Sidorov", entry.Value.Name);
+                        Assert.AreEqual(50, entry.Value.Age);
+                    }
+                    else
+                        Assert.Fail("Unexpected value: " + entry);
+                }
+
+                Assert.IsTrue(first && second && third);
+            }
+        }
+
+        /// <summary>
+        /// Check SQL query.
+        /// </summary>
+        [Test]
+        public void TestSqlQuery()
+        {
+            CheckSqlQuery(MaxItemCnt, false, false);
+        }
+
+        /// <summary>
+        /// Check SQL query in portable mode.
+        /// </summary>
+        [Test]
+        public void TestSqlQueryPortable()
+        {
+            CheckSqlQuery(MaxItemCnt, false, true);
+        }
+
+        /// <summary>
+        /// Check local SQL query.
+        /// </summary>
+        [Test]
+        public void TestSqlQueryLocal()
+        {
+            CheckSqlQuery(MaxItemCnt, true, false);
+        }
+
+        /// <summary>
+        /// Check local SQL query in portable mode.
+        /// </summary>
+        [Test]
+        public void TestSqlQueryLocalPortable()
+        {
+            CheckSqlQuery(MaxItemCnt, true, true);
+        }
+
+        /// <summary>
+        /// Check SQL query.
+        /// </summary>
+        /// <param name="cnt">Amount of cache entries to create.</param>
+        /// <param name="loc">Local query flag.</param>
+        /// <param name="keepPortable">Keep portable flag.</param>
+        private void CheckSqlQuery(int cnt, bool loc, bool keepPortable)
+        {
+            var cache = Cache();
+
+            // 1. Populate cache with data, calculating expected count in parallel.
+            var exp = PopulateCache(cache, loc, cnt, x => x < 50);
+
+            // 2. Validate results.
+            SqlQuery qry = loc ?  new SqlQuery(typeof(QueryPerson), "age < 50", true) :
+                new SqlQuery(typeof(QueryPerson), "age < 50");
+
+            ValidateQueryResults(cache, qry, exp, keepPortable);
+        }
+
+        /// <summary>
+        /// Check SQL fields query.
+        /// </summary>
+        [Test]
+        public void TestSqlFieldsQuery()
+        {
+            CheckSqlFieldsQuery(MaxItemCnt, false);
+        }
+
+        /// <summary>
+        /// Check local SQL fields query.
+        /// </summary>
+        [Test]
+        public void TestSqlFieldsQueryLocal()
+        {
+            CheckSqlFieldsQuery(MaxItemCnt, true);
+        }
+
+        /// <summary>
+        /// Check SQL fields query.
+        /// </summary>
+        /// <param name="cnt">Amount of cache entries to create.</param>
+        /// <param name="loc">Local query flag.</param>
+        private void CheckSqlFieldsQuery(int cnt, bool loc)
+        {
+            var cache = Cache();
+
+            // 1. Populate cache with data, calculating expected count in parallel.
+            var exp = PopulateCache(cache, loc, cnt, x => x < 50);
+
+            // 2. Vlaidate results.
+            SqlFieldsQuery qry = loc ? new SqlFieldsQuery("SELECT name, age FROM QueryPerson WHERE age < 50", true) :
+                new SqlFieldsQuery("SELECT name, age FROM QueryPerson WHERE age < 50");
+
+            using (IQueryCursor<IList> cursor = cache.QueryFields(qry))
+            {
+                HashSet<int> exp0 = new HashSet<int>(exp);
+
+                foreach (var entry in cursor.GetAll())
+                {
+                    Assert.AreEqual(2, entry.Count);
+                    Assert.AreEqual(entry[0].ToString(), entry[1].ToString());
+
+                    exp0.Remove((int)entry[1]);
+                }
+
+                Assert.AreEqual(0, exp0.Count);
+            }
+
+            using (IQueryCursor<IList> cursor = cache.QueryFields(qry))
+            {
+                HashSet<int> exp0 = new HashSet<int>(exp);
+
+                foreach (var entry in cursor)
+                {
+                    Assert.AreEqual(entry[0].ToString(), entry[1].ToString());
+
+                    exp0.Remove((int)entry[1]);
+                }
+
+                Assert.AreEqual(0, exp0.Count);
+            }
+        }
+
+        /// <summary>
+        /// Check text query.
+        /// </summary>
+        [Test]
+        public void TestTextQuery()
+        {
+            CheckTextQuery(MaxItemCnt, false, false);
+        }
+
+        /// <summary>
+        /// Check SQL query in portable mode.
+        /// </summary>
+        [Test]
+        public void TestTextQueryPortable()
+        {
+            CheckTextQuery(MaxItemCnt, false, true);
+        }
+
+        /// <summary>
+        /// Check local SQL query.
+        /// </summary>
+        [Test]
+        public void TestTextQueryLocal()
+        {
+            CheckTextQuery(MaxItemCnt, true, false);
+        }
+
+        /// <summary>
+        /// Check local SQL query in portable mode.
+        /// </summary>
+        [Test]
+        public void TestTextQueryLocalPortable()
+        {
+            CheckTextQuery(MaxItemCnt, true, true);
+        }
+
+        /// <summary>
+        /// Check text query.
+        /// </summary>
+        /// <param name="cnt">Amount of cache entries to create.</param>
+        /// <param name="loc">Local query flag.</param>
+        /// <param name="keepPortable">Keep portable flag.</param>
+        private void CheckTextQuery(int cnt, bool loc, bool keepPortable)
+        {
+            var cache = Cache();
+
+            // 1. Populate cache with data, calculating expected count in parallel.
+            var exp = PopulateCache(cache, loc, cnt, x => x.ToString().StartsWith("1"));
+
+            // 2. Validate results.
+            TextQuery qry = loc ? new TextQuery(typeof(QueryPerson), "1*", true) :
+                new TextQuery(typeof(QueryPerson), "1*");
+
+            ValidateQueryResults(cache, qry, exp, keepPortable);
+        }
+
+        /// <summary>
+        /// Check scan query.
+        /// </summary>
+        [Test]
+        public void TestScanQuery()
+        {
+            CheckScanQuery<QueryPerson>(MaxItemCnt, false, false);
+        }
+
+        /// <summary>
+        /// Check scan query in portable mode.
+        /// </summary>
+        [Test]
+        public void TestScanQueryPortable()
+        {
+            CheckScanQuery<PortableUserObject>(MaxItemCnt, false, true);
+        }
+
+        /// <summary>
+        /// Check local scan query.
+        /// </summary>
+        [Test]
+        public void TestScanQueryLocal()
+        {
+            CheckScanQuery<QueryPerson>(MaxItemCnt, true, false);
+        }
+
+        /// <summary>
+        /// Check local scan query in portable mode.
+        /// </summary>
+        [Test]
+        public void TestScanQueryLocalPortable()
+        {
+            CheckScanQuery<PortableUserObject>(MaxItemCnt, true, true);
+        }
+
+        /// <summary>
+        /// Check scan query with partitions.
+        /// </summary>
+        [Test]
+        [Ignore("IGNITE-1012")]
+        public void TestScanQueryPartitions([Values(true, false)]  bool loc)
+        {
+            CheckScanQueryPartitions<QueryPerson>(MaxItemCnt, loc, false);
+        }
+
+        /// <summary>
+        /// Check scan query with partitions in portable mode.
+        /// </summary>
+        [Test]
+        [Ignore("IGNITE-1012")]
+        public void TestScanQueryPartitionsPortable([Values(true, false)]  bool loc)
+        {
+            CheckScanQueryPartitions<PortableUserObject>(MaxItemCnt, loc, true);
+        }
+
+        /// <summary>
+        /// Tests that query attempt on non-indexed cache causes an exception.
+        /// </summary>
+        [Test]
+        public void TestIndexingDisabledError()
+        {
+            var cache = GetIgnite(0).GetOrCreateCache<int, QueryPerson>("nonindexed_cache");
+
+            var queries = new QueryBase[]
+            {
+                new TextQuery(typeof (QueryPerson), "1*"),
+                new SqlQuery(typeof (QueryPerson), "age < 50")
+            };
+
+            foreach (var qry in queries)
+            {
+                var err = Assert.Throws<IgniteException>(() => cache.Query(qry));
+
+                Assert.AreEqual("Indexing is disabled for cache: nonindexed_cache. " +
+                    "Use setIndexedTypes or setTypeMetadata methods on CacheConfiguration to enable.", err.Message);
+            }
+        }
+
+        /// <summary>
+        /// Check scan query.
+        /// </summary>
+        /// <param name="cnt">Amount of cache entries to create.</param>
+        /// <param name="loc">Local query flag.</param>
+        /// <param name="keepPortable">Keep portable flag.</param>
+        private void CheckScanQuery<TV>(int cnt, bool loc, bool keepPortable)
+        {
+            var cache = Cache();
+
+            // No predicate
+            var exp = PopulateCache(cache, loc, cnt, x => true);
+            var qry = new ScanQuery<int, TV>();
+            ValidateQueryResults(cache, qry, exp, keepPortable);
+
+            // Serializable
+            exp = PopulateCache(cache, loc, cnt, x => x < 50);
+            qry = new ScanQuery<int, TV>(new ScanQueryFilter<TV>());
+            ValidateQueryResults(cache, qry, exp, keepPortable);
+
+            // Portable
+            exp = PopulateCache(cache, loc, cnt, x => x < 50);
+            qry = new ScanQuery<int, TV>(new PortableScanQueryFilter<TV>());
+            ValidateQueryResults(cache, qry, exp, keepPortable);
+
+            // Exception
+            exp = PopulateCache(cache, loc, cnt, x => x < 50);
+            qry = new ScanQuery<int, TV>(new ScanQueryFilter<TV> {ThrowErr = true});
+            
+            var ex = Assert.Throws<IgniteException>(() => ValidateQueryResults(cache, qry, exp, keepPortable));
+            Assert.AreEqual(ScanQueryFilter<TV>.ErrMessage, ex.Message);
+        }
+
+        /// <summary>
+        /// Checks scan query with partitions.
+        /// </summary>
+        /// <param name="cnt">Amount of cache entries to create.</param>
+        /// <param name="loc">Local query flag.</param>
+        /// <param name="keepPortable">Keep portable flag.</param>
+        private void CheckScanQueryPartitions<TV>(int cnt, bool loc, bool keepPortable)
+        {
+            StopGrids();
+            StartGrids();
+
+            var cache = Cache();
+
+            var aff = cache.Ignite.Affinity(CacheName);
+            var exp = PopulateCache(cache, loc, cnt, x => true);  // populate outside the loop (slow)
+
+            for (var part = 0; part < aff.Partitions; part++)
+            {
+                //var exp0 = new HashSet<int>(exp.Where(x => aff.Partition(x) == part)); // filter expected keys
+                var exp0 = new HashSet<int>();
+                foreach (var x in exp)
+                    if (aff.Partition(x) == part)
+                        exp0.Add(x);
+
+                var qry = new ScanQuery<int, TV> { Partition = part };
+
+                Console.WriteLine("Checking query on partition " + part);
+                ValidateQueryResults(cache, qry, exp0, keepPortable);
+            }
+
+            // Partitions with predicate
+            exp = PopulateCache(cache, loc, cnt, x => x < 50);  // populate outside the loop (slow)
+
+            for (var part = 0; part < aff.Partitions; part++)
+            {
+                //var exp0 = new HashSet<int>(exp.Where(x => aff.Partition(x) == part)); // filter expected keys
+                var exp0 = new HashSet<int>();
+                foreach (var x in exp)
+                    if (aff.Partition(x) == part)
+                        exp0.Add(x);
+
+                var qry = new ScanQuery<int, TV>(new ScanQueryFilter<TV>()) { Partition = part };
+
+                Console.WriteLine("Checking predicate query on partition " + part);
+                ValidateQueryResults(cache, qry, exp0, keepPortable);
+            }
+            
+        }
+
+        /// <summary>
+        /// Validates the query results.
+        /// </summary>
+        /// <param name="cache">Cache.</param>
+        /// <param name="qry">Query.</param>
+        /// <param name="exp">Expected keys.</param>
+        /// <param name="keepPortable">Keep portable flag.</param>
+        private static void ValidateQueryResults(ICache<int, QueryPerson> cache, QueryBase qry, HashSet<int> exp,
+            bool keepPortable)
+        {
+            if (keepPortable)
+            {
+                var cache0 = cache.WithKeepPortable<int, IPortableObject>();
+
+                using (var cursor = cache0.Query(qry))
+                {
+                    HashSet<int> exp0 = new HashSet<int>(exp);
+                    var all = new List<ICacheEntry<int, object>>();
+
+                    foreach (var entry in cursor.GetAll())
+                    {
+                        all.Add(entry);
+
+                        Assert.AreEqual(entry.Key.ToString(), entry.Value.Field<string>("name"));
+                        Assert.AreEqual(entry.Key, entry.Value.Field<int>("age"));
+
+                        exp0.Remove(entry.Key);
+                    }
+
+                    AssertMissingExpectedKeys(exp0, cache, all);
+                }
+
+                using (var cursor = cache0.Query(qry))
+                {
+                    HashSet<int> exp0 = new HashSet<int>(exp);
+                    var all = new List<ICacheEntry<int, object>>();
+
+                    foreach (var entry in cursor)
+                    {
+                        all.Add(entry);
+
+                        Assert.AreEqual(entry.Key.ToString(), entry.Value.Field<string>("name"));
+                        Assert.AreEqual(entry.Key, entry.Value.Field<int>("age"));
+
+                        exp0.Remove(entry.Key);
+                    }
+
+                    AssertMissingExpectedKeys(exp0, cache, all);
+                }
+            }
+            else
+            {
+                using (var cursor = cache.Query(qry))
+                {
+                    HashSet<int> exp0 = new HashSet<int>(exp);
+                    var all = new List<ICacheEntry<int, object>>();
+
+                    foreach (var entry in cursor.GetAll())
+                    {
+                        all.Add(entry);
+
+                        Assert.AreEqual(entry.Key.ToString(), entry.Value.Name);
+                        Assert.AreEqual(entry.Key, entry.Value.Age);
+
+                        exp0.Remove(entry.Key);
+                    }
+
+                    AssertMissingExpectedKeys(exp0, cache, all);
+                }
+
+                using (var cursor = cache.Query(qry))
+                {
+                    HashSet<int> exp0 = new HashSet<int>(exp);
+                    var all = new List<ICacheEntry<int, object>>();
+
+                    foreach (var entry in cursor)
+                    {
+                        all.Add(entry);
+
+                        Assert.AreEqual(entry.Key.ToString(), entry.Value.Name);
+                        Assert.AreEqual(entry.Key, entry.Value.Age);
+
+                        exp0.Remove(entry.Key);
+                    }
+
+                    AssertMissingExpectedKeys(exp0, cache, all);
+                }
+            }
+        }
+
+        /// <summary>
+        /// Asserts that all expected entries have been received.
+        /// </summary>
+        private static void AssertMissingExpectedKeys(ICollection<int> exp, ICache<int, QueryPerson> cache, 
+            IList<ICacheEntry<int, object>> all)
+        {
+            if (exp.Count == 0)
+                return;
+
+            var sb = new StringBuilder();
+            var aff = cache.Ignite.Affinity(cache.Name);
+
+            foreach (var key in exp)
+            {
+                var part = aff.Partition(key);
+                sb.AppendFormat(
+                    "Query did not return expected key '{0}' (exists: {1}), partition '{2}', partition nodes: ", 
+                    key, cache.Get(key) != null, part);
+
+                var partNodes = aff.MapPartitionToPrimaryAndBackups(part);
+
+                foreach (var node in partNodes)
+                    sb.Append(node).Append("  ");
+
+                sb.AppendLine(";");
+            }
+
+            sb.Append("Returned keys: ");
+
+            foreach (var e in all)
+                sb.Append(e.Key).Append(" ");
+
+            sb.AppendLine(";");
+
+            Assert.Fail(sb.ToString());
+        }
+
+        /// <summary>
+        /// Populates the cache with random entries and returns expected results set according to filter.
+        /// </summary>
+        /// <param name="cache">The cache.</param>
+        /// <param name="cnt">Amount of cache entries to create.</param>
+        /// <param name="loc">Local query flag.</param>
+        /// <param name="expectedEntryFilter">The expected entry filter.</param>
+        /// <returns>Expected results set.</returns>
+        private static HashSet<int> PopulateCache(ICache<int, QueryPerson> cache,  bool loc, int cnt,
+            Func<int, bool> expectedEntryFilter)
+        {
+            var rand = new Random();
+
+            var exp = new HashSet<int>();
+
+            for (var i = 0; i < cnt; i++)
+            {
+                var val = rand.Next(100);
+
+                cache.Put(val, new QueryPerson(val.ToString(), val));
+
+                if (expectedEntryFilter(val) && (!loc || cache.Ignite.Affinity(cache.Name)
+                    .IsPrimary(cache.Ignite.Cluster.LocalNode, val)))
+                    exp.Add(val);
+            }
+
+            return exp;
+        }
+    }
+
+    /// <summary>
+    /// Person.
+    /// </summary>
+    public class QueryPerson
+    {
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        public QueryPerson()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="name">Name.</param>
+        /// <param name="age">Age.</param>
+        public QueryPerson(string name, int age)
+        {
+            Name = name;
+            Age = age;
+        }
+
+        /// <summary>
+        /// Name.
+        /// </summary>
+        public string Name { get; set; }
+
+        /// <summary>
+        /// Age.
+        /// </summary>
+        public int Age { get; set; }
+    }
+
+    /// <summary>
+    /// Query filter.
+    /// </summary>
+    [Serializable]
+    public class ScanQueryFilter<TV> : ICacheEntryFilter<int, TV>
+    {
+        // Error message
+        public const string ErrMessage = "Error in ScanQueryFilter.Invoke";
+
+        // Error flag
+        public bool ThrowErr { get; set; }
+
+        /** <inheritdoc /> */
+        public bool Invoke(ICacheEntry<int, TV> entry)
+        {
+            if (ThrowErr)
+                throw new Exception(ErrMessage);
+
+            return entry.Key < 50;
+        }
+    }
+
+    /// <summary>
+    /// Portable query filter.
+    /// </summary>
+    public class PortableScanQueryFilter<TV> : ScanQueryFilter<TV>, IPortableMarshalAware
+    {
+        /** <inheritdoc /> */
+        public void WritePortable(IPortableWriter writer)
+        {
+            var w = writer.RawWriter();
+
+            w.WriteBoolean(ThrowErr);
+        }
+
+        /** <inheritdoc /> */
+        public void ReadPortable(IPortableReader reader)
+        {
+            var r = reader.RawReader();
+
+            ThrowErr = r.ReadBoolean();
+        }
+    }
+}


[10/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedPortableClosureTaskTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedPortableClosureTaskTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedPortableClosureTaskTest.cs
new file mode 100644
index 0000000..4b303ce
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedPortableClosureTaskTest.cs
@@ -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.
+ */
+
+namespace Apache.Ignite.Core.Tests.Compute.Forked
+{
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Forked closure execution tests for portable objects.
+    /// </summary>
+    [Ignore("IGNITE-1367")]
+    public class ForkedPortableClosureTaskTest : PortableClosureTaskTest
+    {
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        public ForkedPortableClosureTaskTest() : base(true) { }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedResourceTaskTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedResourceTaskTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedResourceTaskTest.cs
new file mode 100644
index 0000000..00fd28a
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedResourceTaskTest.cs
@@ -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.
+ */
+
+namespace Apache.Ignite.Core.Tests.Compute.Forked
+{
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Forked resource task test.
+    /// </summary>
+    [Ignore("IGNITE-1367")]
+    public class ForkedResourceTaskTest : ResourceTaskTest
+    {
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        public ForkedResourceTaskTest() : base(true) { }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedSerializableClosureTaskTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedSerializableClosureTaskTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedSerializableClosureTaskTest.cs
new file mode 100644
index 0000000..edc18f2
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedSerializableClosureTaskTest.cs
@@ -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.
+ */
+
+namespace Apache.Ignite.Core.Tests.Compute.Forked
+{
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Forked closure execution tests for serializable objects.
+    /// </summary>
+    [Ignore("IGNITE-1367")]
+    public class ForkedSerializableClosureTaskTest : SerializableClosureTaskTest
+    {
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        public ForkedSerializableClosureTaskTest() : base(true) { }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedTaskAdapterTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedTaskAdapterTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedTaskAdapterTest.cs
new file mode 100644
index 0000000..e7bad17
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedTaskAdapterTest.cs
@@ -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.
+ */
+
+namespace Apache.Ignite.Core.Tests.Compute.Forked
+{
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Forked task adapter test.
+    /// </summary>
+    [Ignore("IGNITE-1367")]
+    public class ForkedTaskAdapterTest : TaskAdapterTest
+    {
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        public ForkedTaskAdapterTest() : base(true) { }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/IgniteExceptionTaskSelfTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/IgniteExceptionTaskSelfTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/IgniteExceptionTaskSelfTest.cs
new file mode 100644
index 0000000..9918dce
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/IgniteExceptionTaskSelfTest.cs
@@ -0,0 +1,753 @@
+/*
+ * 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.Compute
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Linq;
+    using System.Runtime.Serialization;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Resource;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests for exception handling on various task execution stages.
+    /// </summary>
+    public class IgniteExceptionTaskSelfTest : AbstractTaskTest
+    {
+        /** Error mode. */
+        public static ErrorMode Mode;
+
+        /** Observed job errors. */
+        public static readonly ICollection<Exception> JobErrs = new List<Exception>();
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        public IgniteExceptionTaskSelfTest() : base(false) { }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="fork">Fork flag.</param>
+        protected IgniteExceptionTaskSelfTest(bool fork) : base(fork) { }
+
+        /// <summary>
+        /// Test error occurred during map step.
+        /// </summary>
+        [Test]
+        public void TestMapError()
+        {
+            Mode = ErrorMode.MapErr;
+
+            GoodException e = ExecuteWithError() as GoodException;
+
+            Assert.IsNotNull(e);
+
+            Assert.AreEqual(ErrorMode.MapErr, e.Mode);
+        }
+
+        /// <summary>
+        /// Test not-marshalable error occurred during map step.
+        /// </summary>
+        [Test]
+        public void TestMapNotMarshalableError()
+        {
+            Mode = ErrorMode.MapErrNotMarshalable;
+
+            BadException e = ExecuteWithError() as BadException;
+
+            Assert.IsNotNull(e);
+
+            Assert.AreEqual(ErrorMode.MapErrNotMarshalable, e.Mode);
+        }
+
+        /// <summary>
+        /// Test task behavior when job produced by mapper is not marshalable.
+        /// </summary>
+        [Test]
+        public void TestMapNotMarshalableJob()
+        {
+            Mode = ErrorMode.MapJobNotMarshalable;
+
+            SerializationException e = ExecuteWithError() as SerializationException;
+
+            Assert.IsNotNull(e);
+        }
+
+        /// <summary>
+        /// Test local job error.
+        /// </summary>
+        [Test]
+        public void TestLocalJobError()
+        {
+            Mode = ErrorMode.LocJobErr;
+
+            int res = Execute();
+
+            Assert.AreEqual(2, res);
+
+            Assert.AreEqual(1, JobErrs.Count);
+            Assert.IsNotNull(JobErrs.First() as GoodException);
+            Assert.AreEqual(ErrorMode.LocJobErr, ((GoodException) JobErrs.First()).Mode);
+        }
+
+        /// <summary>
+        /// Test local not-marshalable job error.
+        /// </summary>
+        [Test]
+        public void TestLocalJobErrorNotMarshalable()
+        {
+            Mode = ErrorMode.LocJobErrNotMarshalable;
+
+            int res = Execute();
+
+            Assert.AreEqual(2, res);
+
+            Assert.AreEqual(1, JobErrs.Count);
+            Assert.IsNotNull(JobErrs.First() as BadException); // Local job exception is not marshalled.
+        }
+
+        /// <summary>
+        /// Test local not-marshalable job result.
+        /// </summary>
+        [Test]
+        public void TestLocalJobResultNotMarshalable()
+        {
+            Mode = ErrorMode.LocJobResNotMarshalable;
+
+            int res = Execute();
+
+            Assert.AreEqual(3, res); // Local job result is not marshalled.
+
+            Assert.AreEqual(0, JobErrs.Count);
+        }
+
+        /// <summary>
+        /// Test remote job error.
+        /// </summary>
+        [Test]
+        public void TestRemoteJobError()
+        {
+            Mode = ErrorMode.RmtJobErr;
+
+            int res = Execute();
+
+            Assert.AreEqual(1, res);
+
+            Assert.AreEqual(2, JobErrs.Count);
+
+            Assert.IsNotNull(JobErrs.ElementAt(0) as GoodException);
+            Assert.IsNotNull(JobErrs.ElementAt(1) as GoodException);
+
+            Assert.AreEqual(ErrorMode.RmtJobErr, ((GoodException) JobErrs.ElementAt(0)).Mode);
+            Assert.AreEqual(ErrorMode.RmtJobErr, ((GoodException) JobErrs.ElementAt(1)).Mode);
+        }
+
+        /// <summary>
+        /// Test remote not-marshalable job error.
+        /// </summary>
+        [Test]
+        public void TestRemoteJobErrorNotMarshalable()
+        {
+            Mode = ErrorMode.RmtJobErrNotMarshalable;
+
+            int res = Execute();
+
+            Assert.AreEqual(1, res);
+
+            Assert.AreEqual(2, JobErrs.Count);
+
+            Assert.IsNotNull(JobErrs.ElementAt(0) as IgniteException);
+            Assert.IsNotNull(JobErrs.ElementAt(1) as IgniteException);
+        }
+
+        /// <summary>
+        /// Test local not-marshalable job result.
+        /// </summary>
+        [Test]
+        public void TestRemoteJobResultNotMarshalable()
+        {
+            Mode = ErrorMode.RmtJobResNotMarshalable;
+
+            int res = Execute();
+
+            Assert.AreEqual(1, res);
+
+            Assert.AreEqual(2, JobErrs.Count);
+
+            Assert.IsNotNull(JobErrs.ElementAt(0) as IgniteException);
+            Assert.IsNotNull(JobErrs.ElementAt(1) as IgniteException);
+        }
+
+        /// <summary>
+        /// Test local result error.
+        /// </summary>
+        [Test]
+        public void TestLocalResultError()
+        {
+            Mode = ErrorMode.LocResErr;
+
+            GoodException e = ExecuteWithError() as GoodException;
+
+            Assert.IsNotNull(e);
+
+            Assert.AreEqual(ErrorMode.LocResErr, e.Mode);
+        }
+
+        /// <summary>
+        /// Test local result not-marshalable error.
+        /// </summary>
+        [Test]
+        public void TestLocalResultErrorNotMarshalable()
+        {
+            Mode = ErrorMode.LocResErrNotMarshalable;
+
+            BadException e = ExecuteWithError() as BadException;
+
+            Assert.IsNotNull(e);
+
+            Assert.AreEqual(ErrorMode.LocResErrNotMarshalable, e.Mode);
+        }
+
+        /// <summary>
+        /// Test remote result error.
+        /// </summary>
+        [Test]
+        public void TestRemoteResultError()
+        {
+            Mode = ErrorMode.RmtResErr;
+
+            GoodException e = ExecuteWithError() as GoodException;
+
+            Assert.IsNotNull(e);
+
+            Assert.AreEqual(ErrorMode.RmtResErr, e.Mode);
+        }
+
+        /// <summary>
+        /// Test remote result not-marshalable error.
+        /// </summary>
+        [Test]
+        public void TestRemoteResultErrorNotMarshalable()
+        {
+            Mode = ErrorMode.RmtResErrNotMarshalable;
+
+            BadException e = ExecuteWithError() as BadException;
+
+            Assert.IsNotNull(e);
+
+            Assert.AreEqual(ErrorMode.RmtResErrNotMarshalable, e.Mode);
+        }
+
+        /// <summary>
+        /// Test reduce with error.
+        /// </summary>
+        [Test]
+        public void TestReduceError()
+        {
+            Mode = ErrorMode.ReduceErr;
+
+            GoodException e = ExecuteWithError() as GoodException;
+
+            Assert.IsNotNull(e);
+
+            Assert.AreEqual(ErrorMode.ReduceErr, e.Mode);
+        }
+
+        /// <summary>
+        /// Test reduce with not-marshalable error.
+        /// </summary>
+        [Test]
+        public void TestReduceErrorNotMarshalable()
+        {
+            Mode = ErrorMode.ReduceErrNotMarshalable;
+
+            BadException e = ExecuteWithError() as BadException;
+
+            Assert.IsNotNull(e);
+
+            Assert.AreEqual(ErrorMode.ReduceErrNotMarshalable, e.Mode);
+        }
+
+        /// <summary>
+        /// Test reduce with not-marshalable result.
+        /// </summary>
+        [Test]
+        public void TestReduceResultNotMarshalable()
+        {
+            Mode = ErrorMode.ReduceResNotMarshalable;
+
+            int res = Execute();
+
+            Assert.AreEqual(3, res);
+        }
+
+        /// <summary>
+        /// Execute task successfully.
+        /// </summary>
+        /// <returns>Task result.</returns>
+        private int Execute()
+        {
+            JobErrs.Clear();
+
+            object res = Grid1.Compute().Execute(new Task());
+
+            return res is GoodTaskResult ? ((GoodTaskResult)res).Res : ((BadTaskResult)res).Res;
+        }
+
+        /// <summary>
+        /// Execute task with error.
+        /// </summary>
+        /// <returns>Task</returns>
+        private Exception ExecuteWithError()
+        {
+            JobErrs.Clear();
+
+            Exception err = null;
+
+            try
+            {
+                Grid1.Compute().Execute(new Task());
+
+                Assert.Fail();
+            }
+            catch (Exception e)
+            {
+                err = e;
+            }
+
+            return err;
+        }
+
+        /// <summary>
+        /// Error modes.
+        /// </summary>
+        public enum ErrorMode
+        {
+            /** Error during map step. */
+            MapErr,
+
+            /** Error during map step which is not marshalable. */
+            MapErrNotMarshalable,
+
+            /** Job created by mapper is not marshalable. */
+            MapJobNotMarshalable,
+
+            /** Error occurred in local job. */
+            LocJobErr,
+
+            /** Error occurred in local job and is not marshalable. */
+            LocJobErrNotMarshalable,
+
+            /** Local job result is not marshalable. */
+            LocJobResNotMarshalable,
+
+            /** Error occurred in remote job. */
+            RmtJobErr,
+
+            /** Error occurred in remote job and is not marshalable. */
+            RmtJobErrNotMarshalable,
+
+            /** Remote job result is not marshalable. */
+            RmtJobResNotMarshalable,            
+
+            /** Error occurred during local result processing. */
+            LocResErr,
+
+            /** Error occurred during local result processing and is not marshalable. */
+            LocResErrNotMarshalable,
+
+            /** Error occurred during remote result processing. */
+            RmtResErr,
+
+            /** Error occurred during remote result processing and is not marshalable. */
+            RmtResErrNotMarshalable,
+
+            /** Error during reduce step. */
+            ReduceErr,
+
+            /** Error during reduce step and is not marshalable. */
+            ReduceErrNotMarshalable,
+
+            /** Reduce result is not marshalable. */
+            ReduceResNotMarshalable
+        }
+
+        /// <summary>
+        /// Task.
+        /// </summary>
+        public class Task : IComputeTask<object, object>
+        {
+            /** Grid. */
+            [InstanceResource]
+            private IIgnite _grid = null;
+
+            /** Result. */
+            private int _res;
+
+            /** <inheritDoc /> */
+            public IDictionary<IComputeJob<object>, IClusterNode> Map(IList<IClusterNode> subgrid, object arg)
+            {
+                switch (Mode)
+                {
+                    case ErrorMode.MapErr:
+                        throw new GoodException(ErrorMode.MapErr);
+
+                    case ErrorMode.MapErrNotMarshalable:
+                        throw new BadException(ErrorMode.MapErrNotMarshalable);
+
+                    case ErrorMode.MapJobNotMarshalable:
+                    {
+                        var badJobs = new Dictionary<IComputeJob<object>, IClusterNode>();
+
+                        foreach (IClusterNode node in subgrid)
+                            badJobs.Add(new BadJob(), node);
+
+                        return badJobs;
+                    }
+                }
+
+                // Map completes sucessfully and we spread jobs to all nodes.
+                var jobs = new Dictionary<IComputeJob<object>, IClusterNode>();
+
+                foreach (IClusterNode node in subgrid)
+                    jobs.Add(new GoodJob(!_grid.Cluster.LocalNode.Id.Equals(node.Id)), node);
+
+                return jobs;
+            }
+
+            /** <inheritDoc /> */
+            public ComputeJobResultPolicy Result(IComputeJobResult<object> res, IList<IComputeJobResult<object>> rcvd)
+            {
+                if (res.Exception() != null)
+                    JobErrs.Add(res.Exception());
+                else
+                {
+                    object res0 = res.Data();
+
+                    bool rmt = res0 is GoodJobResult ? ((GoodJobResult)res0).Rmt : ((BadJobResult)res0).Rmt;
+
+                    if (rmt)
+                    {
+                        switch (Mode)
+                        {
+                            case ErrorMode.RmtResErr:
+                                throw new GoodException(ErrorMode.RmtResErr);
+
+                            case ErrorMode.RmtResErrNotMarshalable:
+                                throw new BadException(ErrorMode.RmtResErrNotMarshalable);
+                        }
+                    }
+                    else
+                    {
+                        switch (Mode)
+                        {
+                            case ErrorMode.LocResErr:
+                                throw new GoodException(ErrorMode.LocResErr);
+
+                            case ErrorMode.LocResErrNotMarshalable:
+                                throw new BadException(ErrorMode.LocResErrNotMarshalable);
+                        }
+                    }
+
+                    _res += 1;
+                }
+
+                return ComputeJobResultPolicy.Wait;
+            }
+
+            /** <inheritDoc /> */
+            public object Reduce(IList<IComputeJobResult<object>> results)
+            {
+                switch (Mode)
+                {
+                    case ErrorMode.ReduceErr:
+                        throw new GoodException(ErrorMode.ReduceErr);
+
+                    case ErrorMode.ReduceErrNotMarshalable:
+                        throw new BadException(ErrorMode.ReduceErrNotMarshalable);
+
+                    case ErrorMode.ReduceResNotMarshalable:
+                        return new BadTaskResult(_res);
+                }
+
+                return new GoodTaskResult(_res);
+            }
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        [Serializable]
+        public class GoodJob : IComputeJob<object>
+        {
+            /** Whether the job is remote. */
+            private bool _rmt;
+
+            /// <summary>
+            /// 
+            /// </summary>
+            /// <param name="rmt"></param>
+            public GoodJob(bool rmt)
+            {
+                _rmt = rmt;
+            }
+
+            /// <summary>
+            /// 
+            /// </summary>
+            /// <param name="info"></param>
+            /// <param name="context"></param>
+            public GoodJob(SerializationInfo info, StreamingContext context)
+            {
+                _rmt = info.GetBoolean("rmt");
+            }
+
+            /** <inheritDoc /> */
+            public void GetObjectData(SerializationInfo info, StreamingContext context)
+            {
+                info.AddValue("rmt", _rmt);
+            }
+
+            /** <inheritDoc /> */
+            public object Execute()
+            {
+                if (_rmt)
+                {
+                    switch (Mode)
+                    {
+                        case ErrorMode.RmtJobErr:
+                            throw new GoodException(ErrorMode.RmtJobErr);
+
+                        case ErrorMode.RmtJobErrNotMarshalable:
+                            throw new BadException(ErrorMode.RmtJobErr);
+
+                        case ErrorMode.RmtJobResNotMarshalable:
+                            return new BadJobResult(_rmt);
+                    }
+                }
+                else
+                {
+                    switch (Mode)
+                    {
+                        case ErrorMode.LocJobErr:
+                            throw new GoodException(ErrorMode.LocJobErr);
+
+                        case ErrorMode.LocJobErrNotMarshalable:
+                            throw new BadException(ErrorMode.LocJobErr);
+
+                        case ErrorMode.LocJobResNotMarshalable:
+                            return new BadJobResult(_rmt);
+                    }
+                }
+
+                return new GoodJobResult(_rmt);
+            }
+
+            /** <inheritDoc /> */
+            public void Cancel()
+            {
+                // No-op.
+            }
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        public class BadJob : IComputeJob<object>
+        {
+            [InstanceResource]
+
+            /** <inheritDoc /> */
+            public object Execute()
+            {
+                throw new NotImplementedException();
+            }
+
+            /** <inheritDoc /> */
+            public void Cancel()
+            {
+                // No-op.
+            }
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        [Serializable]
+        public class GoodJobResult
+        {
+            /** */
+            public bool Rmt;
+            
+            /// <summary>
+            /// 
+            /// </summary>
+            /// <param name="rmt"></param>
+            public GoodJobResult(bool rmt)
+            {
+                Rmt = rmt;
+            }
+
+            /// <summary>
+            /// 
+            /// </summary>
+            /// <param name="info"></param>
+            /// <param name="context"></param>
+            public GoodJobResult(SerializationInfo info, StreamingContext context)
+            {
+                Rmt = info.GetBoolean("rmt");
+            }
+
+            /** <inheritDoc /> */
+            public void GetObjectData(SerializationInfo info, StreamingContext context)
+            {
+                info.AddValue("rmt", Rmt);
+            }
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        public class BadJobResult
+        {
+            /** */
+            public bool Rmt;
+
+            /// <summary>
+            /// 
+            /// </summary>
+            /// <param name="rmt"></param>
+            public BadJobResult(bool rmt)
+            {
+                Rmt = rmt;
+            }
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        [Serializable]
+        public class GoodTaskResult
+        {
+            /** */
+            public int Res;
+
+            /// <summary>
+            /// 
+            /// </summary>
+            /// <param name="res"></param>
+            public GoodTaskResult(int res)
+            {
+                Res = res;
+            }
+
+            /// <summary>
+            /// 
+            /// </summary>
+            /// <param name="info"></param>
+            /// <param name="context"></param>
+            public GoodTaskResult(SerializationInfo info, StreamingContext context)
+            {
+                Res = info.GetInt32("res");
+            }
+
+            /** <inheritDoc /> */
+            public void GetObjectData(SerializationInfo info, StreamingContext context)
+            {
+                info.AddValue("res", Res);
+            }
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        public class BadTaskResult
+        {
+            /** */
+            public int Res;
+
+            /// <summary>
+            /// 
+            /// </summary>
+            /// <param name="res"></param>
+            public BadTaskResult(int res)
+            {
+                Res = res;
+            }
+        }
+
+        /// <summary>
+        /// Marshalable exception.
+        /// </summary>
+        [Serializable]
+        public class GoodException : Exception
+        {
+            /** */
+            public ErrorMode Mode;
+            
+            /// <summary>
+            /// 
+            /// </summary>
+            /// <param name="mode"></param>
+            public GoodException(ErrorMode mode)
+            {
+                Mode = mode;
+            }
+
+            /// <summary>
+            /// 
+            /// </summary>
+            /// <param name="info"></param>
+            /// <param name="context"></param>
+            public GoodException(SerializationInfo info, StreamingContext context)
+            {
+                Mode = (ErrorMode)info.GetInt32("mode");
+            }
+
+            /** <inheritDoc /> */
+            public override void GetObjectData(SerializationInfo info, StreamingContext context)
+            {
+                info.AddValue("mode", (int)Mode);
+
+                base.GetObjectData(info, context);
+            }
+        }
+
+        /// <summary>
+        /// Not marshalable exception.
+        /// </summary>
+        public class BadException : Exception
+        {
+            /** */
+            public ErrorMode Mode;
+
+            /// <summary>
+            /// 
+            /// </summary>
+            /// <param name="mode"></param>
+            public BadException(ErrorMode mode)
+            {
+                Mode = mode;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/PortableClosureTaskTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/PortableClosureTaskTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/PortableClosureTaskTest.cs
new file mode 100644
index 0000000..3ca933e
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/PortableClosureTaskTest.cs
@@ -0,0 +1,217 @@
+/*
+ * 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.Compute
+{
+    using System;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Portable;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Closure execution tests for portable objects.
+    /// </summary>
+    public class PortableClosureTaskTest : ClosureTaskTest
+    {
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        public PortableClosureTaskTest() : base(false) { }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="fork">Fork flag.</param>
+        protected PortableClosureTaskTest(bool fork) : base(fork) { }
+
+        /** <inheritDoc /> */
+        protected override void PortableTypeConfigurations(ICollection<PortableTypeConfiguration> portTypeCfgs)
+        {
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(PortableOutFunc)));
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(PortableFunc)));
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(PortableResult)));
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(PortableException)));
+        }
+
+        /** <inheritDoc /> */
+        protected override IComputeFunc<object> OutFunc(bool err)
+        {
+            return new PortableOutFunc(err);
+        }
+
+        /** <inheritDoc /> */
+        protected override IComputeFunc<object, object> Func(bool err)
+        {
+            return new PortableFunc(err);
+        }
+
+        /** <inheritDoc /> */
+        protected override void CheckResult(object res)
+        {
+            Assert.IsTrue(res != null);
+
+            PortableResult res0 = res as PortableResult;
+
+            Assert.IsTrue(res0 != null);
+            Assert.AreEqual(1, res0.Res);
+        }
+
+        /** <inheritDoc /> */
+        protected override void CheckError(Exception err)
+        {
+            Assert.IsTrue(err != null);
+
+            PortableException err0 = err as PortableException;
+
+            Assert.IsTrue(err0 != null);
+            Assert.AreEqual(ErrMsg, err0.Msg);
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        private class PortableOutFunc : IComputeFunc<object>
+        {
+            /** Error. */
+            private bool _err;
+
+            /// <summary>
+            /// 
+            /// </summary>
+            public PortableOutFunc()
+            {
+                // No-op.
+            }
+
+            /// <summary>
+            /// 
+            /// </summary>
+            /// <param name="err"></param>
+            public PortableOutFunc(bool err)
+            {
+                _err = err;
+            }
+            
+            /** <inheritDoc /> */
+            public object Invoke()
+            {
+                if (_err)
+                    throw new PortableException(ErrMsg);
+                return new PortableResult(1);
+            }
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        private class PortableFunc : IComputeFunc<object, object>
+        {
+            /** Error. */
+            private bool _err;
+
+            /// <summary>
+            /// 
+            /// </summary>
+            public PortableFunc()
+            {
+                // No-op.
+            }
+
+            /// <summary>
+            /// 
+            /// </summary>
+            /// <param name="err"></param>
+            public PortableFunc(bool err)
+            {
+                _err = err;
+            }
+            
+            /** <inheritDoc /> */
+            public object Invoke(object arg)
+            {
+                if (_err)
+                    throw new PortableException(ErrMsg);
+                return new PortableResult(1);
+            }
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        private class PortableException : Exception, IPortableMarshalAware
+        {
+            /** */
+            public string Msg;
+
+            /// <summary>
+            /// 
+            /// </summary>
+            public PortableException()
+            {
+                // No-op.
+            }
+
+            /// <summary>
+            /// 
+            /// </summary>
+            /// <param name="msg"></param>
+            public PortableException(string msg) : this()
+            {
+                Msg = msg;
+            }
+
+            /** <inheritDoc /> */
+            public void WritePortable(IPortableWriter writer)
+            {
+                writer.RawWriter().WriteString(Msg);
+            }
+
+            /** <inheritDoc /> */
+            public void ReadPortable(IPortableReader reader)
+            {
+                Msg = reader.RawReader().ReadString();
+            }
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        private class PortableResult
+        {
+            /** */
+            public int Res;
+
+            /// <summary>
+            /// 
+            /// </summary>
+            public PortableResult()
+            {
+                // No-op.
+            }
+
+            /// <summary>
+            /// 
+            /// </summary>
+            /// <param name="res"></param>
+            public PortableResult(int res)
+            {
+                Res = res;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/PortableTaskTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/PortableTaskTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/PortableTaskTest.cs
new file mode 100644
index 0000000..b3bd1b1
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/PortableTaskTest.cs
@@ -0,0 +1,253 @@
+/*
+ * 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.Compute
+{
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Portable;
+    using Apache.Ignite.Core.Resource;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Task test result.
+    /// </summary>
+    public class PortableTaskTest : AbstractTaskTest
+    {
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        public PortableTaskTest() : base(false) { }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="fork">Fork flag.</param>
+        protected PortableTaskTest(bool fork) : base(fork) { }
+
+        /// <summary>
+        /// Test for task result.
+        /// </summary>
+        [Test]
+        public void TestPortableObjectInTask()
+        {
+            IPortableObject taskArg = ToPortable(Grid1, new PortableTaskArgument(100));
+
+            TestTask task = new TestTask(Grid1, taskArg);
+
+            IPortableObject res = Grid1.Compute().Execute(task, taskArg);
+
+            Assert.NotNull(res);
+
+            Assert.AreEqual(400, res.Field<int>("val"));
+
+            PortableTaskResult resObj = res.Deserialize<PortableTaskResult>();
+
+            Assert.AreEqual(400, resObj.Val);
+        }
+
+        private static IPortableObject ToPortable(IIgnite grid, object obj)
+        {
+            var cache = grid.Cache<object, object>(Cache1Name).WithKeepPortable<object, object>();
+
+            cache.Put(1, obj);
+
+            return (IPortableObject) cache.Get(1);
+        }
+
+        /** <inheritDoc /> */
+        override protected void PortableTypeConfigurations(ICollection<PortableTypeConfiguration> portTypeCfgs)
+        {
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(PortableJobArgument)));
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(PortableJobResult)));
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(PortableTaskArgument)));
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(PortableTaskResult)));
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(PortableJob)));
+        }
+
+        /// <summary>
+        /// Test task.
+        /// </summary>
+        public class TestTask : ComputeTaskAdapter<IPortableObject, IPortableObject, IPortableObject>
+        {
+            /** */
+            private readonly IIgnite _grid;
+
+            private readonly IPortableObject _taskArgField;
+
+            public TestTask(IIgnite grid, IPortableObject taskArgField)
+            {
+                _grid = grid;
+                _taskArgField = taskArgField;
+            }
+
+            /** <inheritDoc /> */
+            override public IDictionary<IComputeJob<IPortableObject>, IClusterNode> Map(IList<IClusterNode> subgrid, IPortableObject arg)
+            {
+                Assert.AreEqual(3, subgrid.Count);
+                Assert.NotNull(_grid);
+
+                IPortableObject taskArg = arg;
+
+                CheckTaskArgument(taskArg);
+
+                CheckTaskArgument(_taskArgField);
+
+                IDictionary<IComputeJob<IPortableObject>, IClusterNode> jobs = new Dictionary<IComputeJob<IPortableObject>, IClusterNode>();
+
+
+                foreach (IClusterNode node in subgrid)
+                {
+                    if (!Grid3Name.Equals(node.Attribute<string>("org.apache.ignite.ignite.name"))) // Grid3 does not have cache.
+                    {
+                        PortableJob job = new PortableJob();
+
+                        job.Arg = ToPortable(_grid, new PortableJobArgument(200));
+
+                        jobs.Add(job, node);
+                    }
+                }
+
+                Assert.AreEqual(2, jobs.Count);
+
+                return jobs;
+            }
+
+            private void CheckTaskArgument(IPortableObject taskArg)
+            {
+                Assert.IsNotNull(taskArg);
+
+                Assert.AreEqual(100, taskArg.Field<int>("val"));
+
+                PortableTaskArgument taskArgObj = taskArg.Deserialize<PortableTaskArgument>();
+
+                Assert.AreEqual(100, taskArgObj.Val);
+            }
+
+            /** <inheritDoc /> */
+            override public IPortableObject Reduce(IList<IComputeJobResult<IPortableObject>> results)
+            {
+                Assert.NotNull(_grid);
+
+                Assert.AreEqual(2, results.Count);
+
+                foreach (IComputeJobResult<IPortableObject> res in results)
+                {
+                    IPortableObject jobRes = res.Data();
+
+                    Assert.NotNull(jobRes);
+
+                    Assert.AreEqual(300, jobRes.Field<int>("val"));
+
+                    PortableJobResult jobResObj = jobRes.Deserialize<PortableJobResult>();
+
+                    Assert.AreEqual(300, jobResObj.Val);
+                }
+
+                return ToPortable(_grid, new PortableTaskResult(400));
+            }
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        class PortableJobArgument
+        {
+            /** */
+            public int Val;
+
+            public PortableJobArgument(int val)
+            {
+                Val = val;
+            }
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        class PortableJobResult
+        {
+            /** */
+            public int Val;
+
+            public PortableJobResult(int val)
+            {
+                Val = val;
+            }
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        class PortableTaskArgument
+        {
+            /** */
+            public int Val;
+
+            public PortableTaskArgument(int val)
+            {
+                Val = val;
+            }
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        class PortableTaskResult
+        {
+            /** */
+            public int Val;
+
+            public PortableTaskResult(int val)
+            {
+                Val = val;
+            }
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        class PortableJob : IComputeJob<IPortableObject>
+        {
+            [InstanceResource]
+            private IIgnite _grid = null;
+            
+            /** */
+            public IPortableObject Arg;
+
+            /** <inheritDoc /> */
+            public IPortableObject Execute()
+            {
+                Assert.IsNotNull(Arg);
+
+                Assert.AreEqual(200, Arg.Field<int>("val"));
+
+                PortableJobArgument argObj = Arg.Deserialize<PortableJobArgument>();
+
+                Assert.AreEqual(200, argObj.Val);
+
+                return ToPortable(_grid, new PortableJobResult(300));
+            }
+
+            public void Cancel()
+            {
+                // No-op.
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/ResourceTaskTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/ResourceTaskTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/ResourceTaskTest.cs
new file mode 100644
index 0000000..4cc5982
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/ResourceTaskTest.cs
@@ -0,0 +1,568 @@
+/*
+ * 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.Compute
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Linq;
+    using System.Runtime.Serialization;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Resource;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Test resource injections in tasks and jobs.
+    /// </summary>
+    public class ResourceTaskTest : AbstractTaskTest
+    {
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        public ResourceTaskTest() : base(false) { }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="fork">Fork flag.</param>
+        protected ResourceTaskTest(bool fork) : base(fork) { }
+
+        /// <summary>
+        /// Test Ignite injection into the task.
+        /// </summary>
+        [Test]
+        public void TestTaskInjection()
+        {
+            int res = Grid1.Compute().Execute(new InjectionTask(), 0);
+
+            Assert.AreEqual(Grid1.Cluster.Nodes().Count, res);
+        }
+
+        /// <summary>
+        /// Test Ignite injection into the closure.
+        /// </summary>
+        [Test]
+        public void TestClosureInjection()
+        {
+            var res = Grid1.Compute().Broadcast(new InjectionClosure(), 1);
+
+            Assert.AreEqual(Grid1.Cluster.Nodes().Count, res.Sum());
+        }
+
+        /// <summary>
+        /// Test Ignite injection into reducer.
+        /// </summary>
+        [Test]
+        public void TestReducerInjection()
+        {
+            int res = Grid1.Compute().Apply(new InjectionClosure(), new List<int> { 1, 1, 1 }, new InjectionReducer());
+
+            Assert.AreEqual(Grid1.Cluster.Nodes().Count, res);
+        }
+
+        /// <summary>
+        /// Test no-result-cache attribute.
+        /// </summary>
+        [Test]
+        public void TestNoResultCache()
+        {
+            int res = Grid1.Compute().Execute(new NoResultCacheTask(), 0);
+
+            Assert.AreEqual(Grid1.Cluster.Nodes().Count, res);
+        }
+
+        /// <summary>
+        /// Injection task.
+        /// </summary>
+        public class InjectionTask : Injectee, IComputeTask<object, int, int>
+        {
+            /** <inheritDoc /> */
+            public IDictionary<IComputeJob<int>, IClusterNode> Map(IList<IClusterNode> subgrid, object arg)
+            {
+                CheckInjection();
+
+                return subgrid.ToDictionary(x => (IComputeJob<int>) new InjectionJob(), x => x);
+            }
+
+            /** <inheritDoc /> */
+            public ComputeJobResultPolicy Result(IComputeJobResult<int> res, IList<IComputeJobResult<int>> rcvd)
+            {
+                return ComputeJobResultPolicy.Wait;
+            }
+
+            /** <inheritDoc /> */
+            public int Reduce(IList<IComputeJobResult<int>> results)
+            {
+                return results.Sum(res => res.Data());
+            }
+        }
+
+        /// <summary>
+        /// Injection job.
+        /// </summary>
+        [Serializable]
+        public class InjectionJob : Injectee, IComputeJob<int>
+        {
+            /// <summary>
+            ///
+            /// </summary>
+            public InjectionJob()
+            {
+                // No-op.
+            }
+
+            /// <summary>
+            ///
+            /// </summary>
+            /// <param name="info"></param>
+            /// <param name="context"></param>
+            public InjectionJob(SerializationInfo info, StreamingContext context) : base(info, context)
+            {
+                // No-op.
+            }
+
+            /** <inheritDoc /> */
+            public int Execute()
+            {
+                CheckInjection();
+
+                return 1;
+            }
+
+            public void Cancel()
+            {
+                // No-op.
+            }
+        }
+
+        /// <summary>
+        /// Injection closure.
+        /// </summary>
+        [Serializable]
+        public class InjectionClosure : IComputeFunc<int, int>
+        {
+            /** */
+            [InstanceResource]
+            private static IIgnite _staticGrid1;
+
+            /** */
+            [InstanceResource]
+            public static IIgnite StaticGrid2;
+
+            /// <summary>
+            ///
+            /// </summary>
+            [InstanceResource]
+            public static IIgnite StaticPropGrid1
+            {
+                get { return _staticGrid1; }
+                set { _staticGrid1 = value; }
+            }
+
+            /// <summary>
+            ///
+            /// </summary>
+            [InstanceResource]
+            private static IIgnite StaticPropGrid2
+            {
+                get { return StaticGrid2; }
+                set { StaticGrid2 = value; }
+            }
+
+            /// <summary>
+            ///
+            /// </summary>
+            /// <param name="grid"></param>
+            [InstanceResource]
+            public static void StaticMethod1(IIgnite grid)
+            {
+                _staticGrid1 = grid;
+            }
+
+            /// <summary>
+            ///
+            /// </summary>
+            /// <param name="grid"></param>
+            [InstanceResource]
+            private static void StaticMethod2(IIgnite grid)
+            {
+                StaticGrid2 = grid;
+            }
+
+            /// <summary>
+            ///
+            /// </summary>
+            public InjectionClosure()
+            {
+                // No-op.
+            }
+
+            /// <summary>
+            ///
+            /// </summary>
+            /// <param name="info"></param>
+            /// <param name="context"></param>
+            public InjectionClosure(SerializationInfo info, StreamingContext context)
+            {
+                // No-op.
+            }
+
+            /** */
+            [InstanceResource]
+            private readonly IIgnite _grid1 = null;
+
+            /** */
+            [InstanceResource]
+            public IIgnite Grid2;
+
+            /** */
+            private IIgnite _mthdGrid1;
+
+            /** */
+            private IIgnite _mthdGrid2;
+
+            /// <summary>
+            ///
+            /// </summary>
+            [InstanceResource]
+            public IIgnite PropGrid1
+            {
+                get;
+                set;
+            }
+
+            /// <summary>
+            ///
+            /// </summary>
+            [InstanceResource]
+            private IIgnite PropGrid2
+            {
+                get;
+                set;
+            }
+
+            /// <summary>
+            ///
+            /// </summary>
+            /// <param name="grid"></param>
+            [InstanceResource]
+            public void Method1(IIgnite grid)
+            {
+                _mthdGrid1 = grid;
+            }
+
+            /// <summary>
+            ///
+            /// </summary>
+            /// <param name="grid"></param>
+            [InstanceResource]
+            private void Method2(IIgnite grid)
+            {
+                _mthdGrid2 = grid;
+            }
+
+            /// <summary>
+            /// Check Ignite injections.
+            /// </summary>
+            protected void CheckInjection()
+            {
+                Assert.IsTrue(_staticGrid1 == null);
+                Assert.IsTrue(StaticGrid2 == null);
+
+                Assert.IsTrue(_grid1 != null);
+                Assert.IsTrue(Grid2 == _grid1);
+
+                Assert.IsTrue(PropGrid1 == _grid1);
+                Assert.IsTrue(PropGrid2 == _grid1);
+
+                Assert.IsTrue(_mthdGrid1 == _grid1);
+                Assert.IsTrue(_mthdGrid2 == _grid1);
+            }
+
+            /** <inheritDoc /> */
+            public void GetObjectData(SerializationInfo info, StreamingContext context)
+            {
+                // No-op.
+            }
+
+            /** <inheritDoc /> */
+            public int Invoke(int arg)
+            {
+                CheckInjection();
+
+                return arg;
+            }
+        }
+
+        /// <summary>
+        /// Injection reducer.
+        /// </summary>
+        public class InjectionReducer : Injectee, IComputeReducer<int, int>
+        {
+            /** Collected results. */
+            private readonly ICollection<int> _ress = new List<int>();
+
+            /** <inheritDoc /> */
+            public bool Collect(int res)
+            {
+                CheckInjection();
+
+                lock (_ress)
+                {
+                    _ress.Add(res);
+                }
+
+                return true;
+            }
+
+            /** <inheritDoc /> */
+            public int Reduce()
+            {
+                CheckInjection();
+
+                lock (_ress)
+                {
+                    return _ress.Sum();
+                }
+            }
+        }
+
+        /// <summary>
+        /// Injectee.
+        /// </summary>
+        [Serializable]
+        public class Injectee : ISerializable
+        {
+            /** */
+            [InstanceResource]
+            private static IIgnite _staticGrid1;
+
+            /** */
+            [InstanceResource]
+            public static IIgnite StaticGrid2;
+
+            /// <summary>
+            ///
+            /// </summary>
+            [InstanceResource]
+            public static IIgnite StaticPropGrid1
+            {
+                get { return _staticGrid1; }
+                set { _staticGrid1 = value; }
+            }
+
+            /// <summary>
+            ///
+            /// </summary>
+            [InstanceResource]
+            private static IIgnite StaticPropGrid2
+            {
+                get { return StaticGrid2; }
+                set { StaticGrid2 = value; }
+            }
+
+            /// <summary>
+            ///
+            /// </summary>
+            /// <param name="grid"></param>
+            [InstanceResource]
+            public static void StaticMethod1(IIgnite grid)
+            {
+                _staticGrid1 = grid;
+            }
+
+            /// <summary>
+            ///
+            /// </summary>
+            /// <param name="grid"></param>
+            [InstanceResource]
+            private static void StaticMethod2(IIgnite grid)
+            {
+                StaticGrid2 = grid;
+            }
+
+            /// <summary>
+            ///
+            /// </summary>
+            public Injectee()
+            {
+                // No-op.
+            }
+
+            /// <summary>
+            ///
+            /// </summary>
+            /// <param name="info"></param>
+            /// <param name="context"></param>
+            public Injectee(SerializationInfo info, StreamingContext context)
+            {
+                // No-op.
+            }
+
+            /** */
+            [InstanceResource]
+            private readonly IIgnite _grid1 = null;
+
+            /** */
+            [InstanceResource]
+            public IIgnite Grid2;
+
+            /** */
+            private IIgnite _mthdGrid1;
+
+            /** */
+            private IIgnite _mthdGrid2;
+
+            /// <summary>
+            ///
+            /// </summary>
+            [InstanceResource]
+            public IIgnite PropGrid1
+            {
+                get;
+                set;
+            }
+
+            /// <summary>
+            ///
+            /// </summary>
+            [InstanceResource]
+            private IIgnite PropGrid2
+            {
+                get;
+                set;
+            }
+
+            /// <summary>
+            ///
+            /// </summary>
+            /// <param name="grid"></param>
+            [InstanceResource]
+            public void Method1(IIgnite grid)
+            {
+                _mthdGrid1 = grid;
+            }
+
+            /// <summary>
+            ///
+            /// </summary>
+            /// <param name="grid"></param>
+            [InstanceResource]
+            private void Method2(IIgnite grid)
+            {
+                _mthdGrid2 = grid;
+            }
+
+            /// <summary>
+            /// Check Ignite injections.
+            /// </summary>
+            protected void CheckInjection()
+            {
+                Assert.IsTrue(_staticGrid1 == null);
+                Assert.IsTrue(StaticGrid2 == null);
+
+                Assert.IsTrue(_grid1 != null);
+                Assert.IsTrue(Grid2 == _grid1);
+
+                Assert.IsTrue(PropGrid1 == _grid1);
+                Assert.IsTrue(PropGrid2 == _grid1);
+
+                Assert.IsTrue(_mthdGrid1 == _grid1);
+                Assert.IsTrue(_mthdGrid2 == _grid1);
+            }
+
+            /** <inheritDoc /> */
+            public void GetObjectData(SerializationInfo info, StreamingContext context)
+            {
+                // No-op.
+            }
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        [ComputeTaskNoResultCache]
+        public class NoResultCacheTask : IComputeTask<int, int, int>
+        {
+            /** Sum. */
+            private int _sum;
+
+            /** <inheritDoc /> */
+            public IDictionary<IComputeJob<int>, IClusterNode> Map(IList<IClusterNode> subgrid, int arg)
+            {
+                return subgrid.ToDictionary(x => (IComputeJob<int>) new NoResultCacheJob(), x => x);
+            }
+
+            /** <inheritDoc /> */
+            public ComputeJobResultPolicy Result(IComputeJobResult<int> res, IList<IComputeJobResult<int>> rcvd)
+            {
+                Assert.IsTrue(rcvd != null);
+                Assert.IsTrue(rcvd.Count == 0);
+
+                _sum += res.Data();
+
+                return ComputeJobResultPolicy.Wait;
+            }
+
+            /** <inheritDoc /> */
+            public int Reduce(IList<IComputeJobResult<int>> results)
+            {
+                Assert.IsTrue(results != null);
+                Assert.IsTrue(results.Count == 0);
+
+                return _sum;
+            }
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        [Serializable]
+        public class NoResultCacheJob : IComputeJob<int>
+        {
+            /// <summary>
+            ///
+            /// </summary>
+            public NoResultCacheJob()
+            {
+                // No-op.
+            }
+
+            /// <summary>
+            ///
+            /// </summary>
+            /// <param name="info"></param>
+            /// <param name="context"></param>
+            public NoResultCacheJob(SerializationInfo info, StreamingContext context)
+            {
+                // No-op.
+            }
+
+            /** <inheritDoc /> */
+            public int Execute()
+            {
+                return 1;
+            }
+
+            public void Cancel()
+            {
+                // No-op.
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/SerializableClosureTaskTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/SerializableClosureTaskTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/SerializableClosureTaskTest.cs
new file mode 100644
index 0000000..ded56ed
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/SerializableClosureTaskTest.cs
@@ -0,0 +1,217 @@
+/*
+ * 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.Compute
+{
+    using System;
+    using System.Runtime.Serialization;
+    using Apache.Ignite.Core.Compute;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Closure execution tests for serializable objects.
+    /// </summary>
+    public class SerializableClosureTaskTest : ClosureTaskTest
+    {
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        public SerializableClosureTaskTest() : base(false) { }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="fork">Fork flag.</param>
+        protected SerializableClosureTaskTest(bool fork) : base(fork) { }
+
+        /** <inheritDoc /> */
+        protected override IComputeFunc<object> OutFunc(bool err)
+        {
+            return new SerializableOutFunc(err);
+        }
+
+        /** <inheritDoc /> */
+        protected override IComputeFunc<object, object> Func(bool err)
+        {
+            return new SerializableFunc(err);
+        }
+
+        /** <inheritDoc /> */
+        protected override void CheckResult(object res)
+        {
+            Assert.IsTrue(res != null);
+
+            SerializableResult res0 = res as SerializableResult;
+
+            Assert.IsTrue(res0 != null);
+            Assert.AreEqual(1, res0.Res);
+        }
+
+        /** <inheritDoc /> */
+        protected override void CheckError(Exception err)
+        {
+            Assert.IsTrue(err != null);
+
+            SerializableException err0 = err as SerializableException;
+
+            Assert.IsTrue(err0 != null);
+            Assert.AreEqual(ErrMsg, err0.Msg);
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        [Serializable]
+        private class SerializableOutFunc : IComputeFunc<object>
+        {
+            /** Error. */
+            private bool _err;
+
+            /// <summary>
+            ///
+            /// </summary>
+            public SerializableOutFunc()
+            {
+                // No-op.
+            }
+
+            /// <summary>
+            ///
+            /// </summary>
+            /// <param name="err"></param>
+            public SerializableOutFunc(bool err)
+            {
+                _err = err;
+            }
+
+            /** <inheritDoc /> */
+            public object Invoke()
+            {
+                if (_err)
+                    throw new SerializableException(ErrMsg);
+                return new SerializableResult(1);
+            }
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        [Serializable]
+        private class SerializableFunc : IComputeFunc<object, object>
+        {
+            /** Error. */
+            private bool _err;
+
+            /// <summary>
+            ///
+            /// </summary>
+            public SerializableFunc()
+            {
+                // No-op.
+            }
+
+            /// <summary>
+            ///
+            /// </summary>
+            /// <param name="err"></param>
+            public SerializableFunc(bool err)
+            {
+                _err = err;
+            }
+
+            /** <inheritDoc /> */
+            public object Invoke(object arg)
+            {
+                Console.WriteLine("INVOKED!");
+
+                if (_err)
+                    throw new SerializableException(ErrMsg);
+                return new SerializableResult(1);
+            }
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        [Serializable]
+        private class SerializableException : Exception
+        {
+            /** */
+            public string Msg;
+
+            /// <summary>
+            ///
+            /// </summary>
+            public SerializableException()
+            {
+                // No-op.
+            }
+
+            /// <summary>
+            ///
+            /// </summary>
+            /// <param name="msg"></param>
+            public SerializableException(string msg) : this()
+            {
+                Msg = msg;
+            }
+            /// <summary>
+            ///
+            /// </summary>
+            /// <param name="info"></param>
+            /// <param name="context"></param>
+            public SerializableException(SerializationInfo info, StreamingContext context) : base(info, context)
+            {
+                Msg = info.GetString("msg");
+            }
+
+            /** <inheritDoc /> */
+            public override void GetObjectData(SerializationInfo info, StreamingContext context)
+            {
+                info.AddValue("msg", Msg);
+
+                base.GetObjectData(info, context);
+            }
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        [Serializable]
+        private class SerializableResult
+        {
+            public int Res;
+
+            /// <summary>
+            ///
+            /// </summary>
+            public SerializableResult()
+            {
+                // No-op.
+            }
+
+            /// <summary>
+            ///
+            /// </summary>
+            /// <param name="res"></param>
+            public SerializableResult(int res)
+            {
+                Res = res;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/TaskAdapterTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/TaskAdapterTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/TaskAdapterTest.cs
new file mode 100644
index 0000000..cf8c663
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/TaskAdapterTest.cs
@@ -0,0 +1,274 @@
+/*
+ * 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.Compute
+{
+    using System;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Portable;
+    using Apache.Ignite.Core.Resource;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Test for task and job adapter.
+    /// </summary>
+    public class TaskAdapterTest : AbstractTaskTest
+    {
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        public TaskAdapterTest() : base(false) { }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="fork">Fork flag.</param>
+        protected TaskAdapterTest(bool fork) : base(fork) { }
+
+        /// <summary>
+        /// Test for task adapter.
+        /// </summary>
+        [Test]
+        public void TestTaskAdapter()
+        {
+            Assert.AreEqual(3, Grid1.Cluster.Nodes().Count);
+
+            HashSet<Guid> allNodes = new HashSet<Guid>(); 
+
+            for (int i = 0; i < 20 && allNodes.Count < 3; i++)
+            {
+                HashSet<Guid> res = Grid1.Compute().Execute(new TestSplitTask(), 1);
+
+                Assert.AreEqual(1, res.Count);
+
+                allNodes.UnionWith(res);
+            }
+
+            Assert.AreEqual(3, allNodes.Count);
+
+            HashSet<Guid> res2 = Grid1.Compute().Execute<int, Guid, HashSet<Guid>>(typeof(TestSplitTask), 3);
+
+            Assert.IsTrue(res2.Count > 0);
+
+            Grid1.Compute().Execute(new TestSplitTask(), 100);
+
+            Assert.AreEqual(3, allNodes.Count);
+        }
+        
+        /// <summary>
+        /// Test for job adapter.
+        /// </summary>
+        [Test]
+        public void TestSerializableJobAdapter()
+        {
+            for (int i = 0; i < 10; i++)
+            {
+                bool res = Grid1.Compute().Execute(new TestJobAdapterTask(), true);
+
+                Assert.IsTrue(res);
+            }
+        }
+
+        /// <summary>
+        /// Test for job adapter.
+        /// </summary>
+        [Test]
+        public void TestPortableJobAdapter()
+        {
+            for (int i = 0; i < 10; i++)
+            {
+                bool res = Grid1.Compute().Execute(new TestJobAdapterTask(), false);
+
+                Assert.IsTrue(res);
+            }
+        }
+
+        /** <inheritDoc /> */
+        override protected void PortableTypeConfigurations(ICollection<PortableTypeConfiguration> portTypeCfgs)
+        {
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(PortableJob)));
+        }
+
+        /// <summary>
+        /// Test task.
+        /// </summary>
+        public class TestSplitTask : ComputeTaskSplitAdapter<int, Guid, HashSet<Guid>>
+        {
+            /** <inheritDoc /> */
+            override protected ICollection<IComputeJob<Guid>> Split(int gridSize, int arg)
+            {
+                Assert.AreEqual(3, gridSize);
+
+                int jobsNum = arg;
+
+                Assert.IsTrue(jobsNum > 0);
+
+                var jobs = new List<IComputeJob<Guid>>(jobsNum);
+
+                for (int i = 0; i < jobsNum; i++)
+                    jobs.Add(new NodeIdJob());
+
+                return jobs;
+            }
+
+            /** <inheritDoc /> */
+            override public HashSet<Guid> Reduce(IList<IComputeJobResult<Guid>> results)
+            {
+                HashSet<Guid> nodes = new HashSet<Guid>();
+
+                foreach (var res in results) {
+                    Guid id = res.Data();
+
+                    Assert.NotNull(id);
+
+                    nodes.Add(id);
+                }
+
+                return nodes;
+            }
+        }
+
+        /// <summary>
+        /// Test task.
+        /// </summary>
+        public class TestJobAdapterTask : ComputeTaskSplitAdapter<bool, bool, bool>
+        {
+            /** <inheritDoc /> */
+            override protected ICollection<IComputeJob<bool>> Split(int gridSize, bool arg)
+            {
+                bool serializable = arg;
+
+                ICollection<IComputeJob<bool>> jobs = new List<IComputeJob<bool>>(1);
+
+                if (serializable)
+                    jobs.Add(new SerializableJob(100, "str"));
+                else
+                    jobs.Add(new PortableJob(100, "str"));
+
+                return jobs;
+            }
+
+            /** <inheritDoc /> */
+            override public bool Reduce(IList<IComputeJobResult<bool>> results)
+            {
+                Assert.AreEqual(1, results.Count);
+
+                Assert.IsTrue(results[0].Data());
+
+                return true;
+            }
+        }
+
+        /// <summary>
+        /// Test job.
+        /// </summary>
+        [Serializable]
+        public class NodeIdJob : IComputeJob<Guid>
+        {
+            [InstanceResource]
+            private IIgnite _grid = null;
+
+            /** <inheritDoc /> */
+            public Guid Execute()
+            {
+                Assert.NotNull(_grid);
+
+                return _grid.Cluster.LocalNode.Id;
+            }
+
+            /** <inheritDoc /> */
+            public void Cancel()
+            {
+                // No-op.
+            }
+        }
+
+        /// <summary>
+        /// Test serializable job.
+        /// </summary>
+        [Serializable]
+        public class SerializableJob : ComputeJobAdapter<bool>
+        {
+            [InstanceResource]
+            private IIgnite _grid = null;
+
+            public SerializableJob(params object[] args) : base(args)
+            { 
+                // No-op.
+            }
+
+            /** <inheritDoc /> */
+            override public bool Execute()
+            {
+                Assert.IsFalse(IsCancelled());
+
+                Cancel();
+
+                Assert.IsTrue(IsCancelled());
+
+                Assert.NotNull(_grid);
+
+                int arg1 = Argument<int>(0);
+
+                Assert.AreEqual(100, arg1);
+
+                string arg2 = Argument<string>(1);
+
+                Assert.AreEqual("str", arg2);
+
+                return true;
+            }
+        }
+
+        /// <summary>
+        /// Test portable job.
+        /// </summary>
+        public class PortableJob : ComputeJobAdapter<bool>
+        {
+            [InstanceResource]
+            private IIgnite _grid = null;
+
+            public PortableJob(params object[] args) : base(args)
+            {
+                // No-op.
+            }
+
+            /** <inheritDoc /> */
+            override public bool Execute()
+            {
+                Assert.IsFalse(IsCancelled());
+
+                Cancel();
+
+                Assert.IsTrue(IsCancelled());
+
+                Assert.NotNull(_grid);
+
+                int arg1 = Argument<int>(0);
+
+                Assert.AreEqual(100, arg1);
+
+                string arg2 = Argument<string>(1);
+
+                Assert.AreEqual("str", arg2);
+
+                return true;
+            }
+        }
+    }
+}


[42/45] ignite git commit: Merge remote-tracking branch 'origin/master'

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


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

Branch: refs/heads/ignite-843
Commit: de4a26efdee6087a4d411a906a7661a5c2cbbe30
Parents: f093960 2aff2ee
Author: Anton Vinogradov <av...@apache.org>
Authored: Fri Sep 4 17:29:30 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Fri Sep 4 17:29:30 2015 +0300

----------------------------------------------------------------------
 examples/config/example-default.xml             |   76 +
 examples/config/example-ignite.xml              |   56 +-
 .../config/portable/example-ignite-portable.xml |   44 +
 .../ignite/examples/portable/Address.java       |   72 +
 .../ignite/examples/portable/Employee.java      |   93 +
 .../ignite/examples/portable/EmployeeKey.java   |   90 +
 .../portable/ExamplePortableNodeStartup.java    |   36 +
 .../ignite/examples/portable/Organization.java  |   93 +
 .../examples/portable/OrganizationType.java     |   32 +
 ...mputeClientPortableTaskExecutionExample.java |  154 +
 .../portable/computegrid/ComputeClientTask.java |  116 +
 .../portable/computegrid/package-info.java      |   21 +
 .../CacheClientPortablePutGetExample.java       |  230 ++
 .../CacheClientPortableQueryExample.java        |  328 ++
 .../portable/datagrid/package-info.java         |   21 +
 .../ignite/examples/portable/package-info.java  |   21 +
 .../java8/datagrid/CacheAffinityExample.java    |   15 +
 .../java8/messaging/MessagingExample.java       |   11 +-
 .../CacheClientPortableExampleTest.java         |   46 +
 .../ComputeClientPortableExampleTest.java       |   37 +
 .../testsuites/IgniteExamplesSelfTestSuite.java |    6 +
 .../java8/examples/BasicExamplesSelfTest.java   |   10 +-
 .../java8/examples/CacheExamplesSelfTest.java   |    8 +-
 .../examples/CheckpointExamplesSelfTest.java    |    8 +-
 .../examples/ClusterGroupExampleSelfTest.java   |    4 +-
 .../examples/ContinuationExamplesSelfTest.java  |    8 +-
 .../ContinuousMapperExamplesSelfTest.java       |    8 +-
 .../examples/DeploymentExamplesSelfTest.java    |    6 +-
 .../java8/examples/EventsExamplesSelfTest.java  |    5 +-
 .../HibernateL2CacheExampleSelfTest.java        |    8 +-
 .../java8/examples/IgfsExamplesSelfTest.java    |    6 +-
 .../examples/LifecycleExamplesSelfTest.java     |    8 +-
 .../examples/MemcacheRestExamplesSelfTest.java  |    4 +-
 .../examples/MessagingExamplesSelfTest.java     |    6 +-
 .../examples/MonteCarloExamplesSelfTest.java    |    8 +-
 .../examples/SpringBeanExamplesSelfTest.java    |    8 +-
 .../java8/examples/TaskExamplesSelfTest.java    |    4 +-
 .../IgniteExamplesJ8SelfTestSuite.java          |   12 +-
 .../configuration/CacheConfiguration.java       |   31 +-
 .../apache/ignite/internal/IgniteKernal.java    |    8 +-
 .../portable/PortableClassDescriptor.java       |   79 +-
 .../internal/portable/PortableContext.java      |  120 +-
 .../internal/portable/PortableWriterExImpl.java |   92 +-
 .../dotnet/PlatformDotNetConfiguration.java     |  119 +
 .../PlatformDotNetPortableConfiguration.java    |  228 ++
 ...PlatformDotNetPortableTypeConfiguration.java |  248 ++
 .../GridPortableMarshallerSelfTest.java         |   68 +-
 .../portable/GridPortableWildcardsSelfTest.java |   60 +-
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |    5 +
 .../testsuites/IgniteCacheTestSuite3.java       |    3 +-
 .../query/h2/opt/GridH2IndexBase.java           |   42 +
 .../query/IgniteSqlSplitterSelfTest.java        |   35 +-
 .../ignite/logger/log4j2/Log4J2Logger.java      |   20 +-
 modules/platform/licenses/apache-2.0.txt        |  202 ++
 modules/platform/pom.xml                        |   14 +
 .../platform/src/main/cpp/common/configure.ac   |    2 +-
 .../src/main/cpp/common/ignite-common.pc.in     |    2 +-
 .../cpp/common/include/ignite/common/exports.h  |    8 +-
 .../cpp/common/include/ignite/common/java.h     |   16 +-
 .../src/main/cpp/common/src/exports.cpp         |   16 +-
 .../platform/src/main/cpp/common/src/java.cpp   |   45 +-
 .../platform/src/main/cpp/core-test/Makefile.am |   49 +
 .../main/cpp/core-test/config/cache-query.xml   |   91 +
 .../main/cpp/core-test/config/cache-test.xml    |  129 +
 .../src/main/cpp/core-test/configure.ac         |   62 +
 .../src/main/cpp/core-test/include/Makefile.am  |   22 +
 .../include/ignite/portable_test_defs.h         |  320 ++
 .../include/ignite/portable_test_utils.h        |  516 +++
 .../cpp/core-test/include/teamcity_messages.h   |   55 +
 .../src/main/cpp/core-test/project/README.TXT   |    1 +
 .../main/cpp/core-test/project/vs/README.TXT    |    1 +
 .../cpp/core-test/project/vs/core-test.vcxproj  |  174 +
 .../project/vs/core-test.vcxproj.filters        |   68 +
 .../main/cpp/core-test/src/cache_query_test.cpp |  651 ++++
 .../src/main/cpp/core-test/src/cache_test.cpp   |  481 +++
 .../main/cpp/core-test/src/concurrent_test.cpp  |  186 +
 .../cpp/core-test/src/handle_registry_test.cpp  |  176 +
 .../main/cpp/core-test/src/ignition_test.cpp    |   97 +
 .../src/portable_reader_writer_raw_test.cpp     | 1532 ++++++++
 .../src/portable_reader_writer_test.cpp         | 1951 ++++++++++
 .../cpp/core-test/src/portable_session_test.cpp |  257 ++
 .../cpp/core-test/src/portable_test_defs.cpp    |   65 +
 .../main/cpp/core-test/src/teamcity_boost.cpp   |  159 +
 .../cpp/core-test/src/teamcity_messages.cpp     |  150 +
 modules/platform/src/main/cpp/core/Makefile.am  |   66 +
 modules/platform/src/main/cpp/core/configure.ac |   62 +
 modules/platform/src/main/cpp/core/ignite.pc.in |    9 +
 .../src/main/cpp/core/include/Makefile.am       |   61 +
 .../main/cpp/core/include/ignite/cache/cache.h  | 1153 ++++++
 .../cpp/core/include/ignite/cache/cache_entry.h |  118 +
 .../core/include/ignite/cache/cache_peek_mode.h |   71 +
 .../cpp/core/include/ignite/cache/query/query.h |   27 +
 .../include/ignite/cache/query/query_argument.h |  125 +
 .../include/ignite/cache/query/query_cursor.h   |  191 +
 .../include/ignite/cache/query/query_scan.h     |  151 +
 .../core/include/ignite/cache/query/query_sql.h |  253 ++
 .../include/ignite/cache/query/query_text.h     |  159 +
 .../src/main/cpp/core/include/ignite/guid.h     |  112 +
 .../src/main/cpp/core/include/ignite/ignite.h   |  154 +
 .../core/include/ignite/ignite_configuration.h  |   92 +
 .../main/cpp/core/include/ignite/ignite_error.h |  260 ++
 .../src/main/cpp/core/include/ignite/ignition.h |  195 +
 .../core/include/ignite/impl/cache/cache_impl.h |  418 +++
 .../ignite/impl/cache/query/query_impl.h        |  115 +
 .../core/include/ignite/impl/handle_registry.h  |  202 ++
 .../include/ignite/impl/ignite_environment.h    |  130 +
 .../cpp/core/include/ignite/impl/ignite_impl.h  |  146 +
 .../core/include/ignite/impl/interop/interop.h  |   25 +
 .../ignite/impl/interop/interop_input_stream.h  |  234 ++
 .../ignite/impl/interop/interop_memory.h        |  280 ++
 .../ignite/impl/interop/interop_output_stream.h |  234 ++
 .../cpp/core/include/ignite/impl/operations.h   |  452 +++
 .../ignite/impl/portable/portable_common.h      |  146 +
 .../ignite/impl/portable/portable_id_resolver.h |  106 +
 .../impl/portable/portable_metadata_handler.h   |  102 +
 .../impl/portable/portable_metadata_manager.h   |  120 +
 .../impl/portable/portable_metadata_snapshot.h  |  122 +
 .../impl/portable/portable_metadata_updater.h   |   53 +
 .../portable/portable_metadata_updater_impl.h   |   65 +
 .../ignite/impl/portable/portable_reader_impl.h | 1130 ++++++
 .../ignite/impl/portable/portable_utils.h       |  344 ++
 .../ignite/impl/portable/portable_writer_impl.h |  859 +++++
 .../cpp/core/include/ignite/portable/portable.h |   29 +
 .../include/ignite/portable/portable_consts.h   |  106 +
 .../ignite/portable/portable_containers.h       |  525 +++
 .../ignite/portable/portable_raw_reader.h       |  324 ++
 .../ignite/portable/portable_raw_writer.h       |  300 ++
 .../include/ignite/portable/portable_reader.h   |  355 ++
 .../include/ignite/portable/portable_type.h     |  293 ++
 .../include/ignite/portable/portable_writer.h   |  335 ++
 .../main/cpp/core/os/linux/include/Makefile.am  |   20 +
 .../core/os/linux/include/ignite/impl/utils.h   |  155 +
 .../main/cpp/core/os/linux/src/impl/utils.cpp   |  439 +++
 .../cpp/core/os/win/include/ignite/impl/utils.h |  155 +
 .../src/main/cpp/core/os/win/src/impl/utils.cpp |  453 +++
 .../src/main/cpp/core/project/README.TXT        |    1 +
 .../src/main/cpp/core/project/vs/README.TXT     |    1 +
 .../src/main/cpp/core/project/vs/core.vcxproj   |  272 ++
 .../cpp/core/project/vs/core.vcxproj.filters    |  246 ++
 .../main/cpp/core/project/vs/core.vcxprojrel    |  272 ++
 modules/platform/src/main/cpp/core/src/guid.cpp |   65 +
 .../platform/src/main/cpp/core/src/ignite.cpp   |   43 +
 .../src/main/cpp/core/src/ignite_error.cpp      |  222 ++
 .../platform/src/main/cpp/core/src/ignition.cpp |  468 +++
 .../main/cpp/core/src/impl/cache/cache_impl.cpp |  388 ++
 .../core/src/impl/cache/query/query_impl.cpp    |  193 +
 .../main/cpp/core/src/impl/handle_registry.cpp  |  234 ++
 .../cpp/core/src/impl/ignite_environment.cpp    |  166 +
 .../src/main/cpp/core/src/impl/ignite_impl.cpp  |   42 +
 .../src/impl/interop/interop_input_stream.cpp   |  215 ++
 .../core/src/impl/interop/interop_memory.cpp    |  182 +
 .../src/impl/interop/interop_output_stream.cpp  |  215 ++
 .../impl/portable/portable_metadata_handler.cpp |   78 +
 .../impl/portable/portable_metadata_manager.cpp |  201 ++
 .../portable/portable_metadata_snapshot.cpp     |   70 +
 .../impl/portable/portable_metadata_updater.cpp |   32 +
 .../portable/portable_metadata_updater_impl.cpp |   94 +
 .../src/impl/portable/portable_reader_impl.cpp  |  683 ++++
 .../core/src/impl/portable/portable_utils.cpp   |  214 ++
 .../src/impl/portable/portable_writer_impl.cpp  |  600 ++++
 .../core/src/portable/portable_containers.cpp   |   76 +
 .../core/src/portable/portable_raw_reader.cpp   |  135 +
 .../core/src/portable/portable_raw_writer.cpp   |  147 +
 .../cpp/core/src/portable/portable_reader.cpp   |  142 +
 .../cpp/core/src/portable/portable_type.cpp     |   51 +
 .../cpp/core/src/portable/portable_writer.cpp   |  154 +
 .../Apache.Ignite.Core.csproj                   |  283 +-
 .../Cache/CacheAtomicUpdateTimeoutException.cs  |   67 +
 .../Cache/CacheEntryProcessorException.cs       |   79 +
 .../Apache.Ignite.Core/Cache/CacheException.cs  |   68 +
 .../Cache/CachePartialUpdateException.cs        |  119 +
 .../Apache.Ignite.Core/Cache/CachePeekMode.cs   |   68 +
 .../Cache/Event/CacheEntryEventType.cs          |   41 +
 .../Cache/Event/ICacheEntryEvent.cs             |   40 +
 .../Cache/Event/ICacheEntryEventFilter.cs       |   31 +
 .../Cache/Event/ICacheEntryEventListener.cs     |   33 +
 .../Cache/Expiry/ExpiryPolicy.cs                |   89 +
 .../Cache/Expiry/IExpiryPolicy.cs               |   59 +
 .../dotnet/Apache.Ignite.Core/Cache/ICache.cs   |  542 +++
 .../Apache.Ignite.Core/Cache/ICacheAffinity.cs  |  161 +
 .../Apache.Ignite.Core/Cache/ICacheEntry.cs     |   37 +
 .../Cache/ICacheEntryFilter.cs                  |   34 +
 .../Cache/ICacheEntryProcessor.cs               |   45 +
 .../Cache/ICacheEntryProcessorResult.cs         |   40 +
 .../Apache.Ignite.Core/Cache/ICacheLock.cs      |   58 +
 .../Apache.Ignite.Core/Cache/ICacheMetrics.cs   |  486 +++
 .../Cache/IMutableCacheEntry.cs                 |   47 +
 .../Cache/Query/Continuous/ContinuousQuery.cs   |  170 +
 .../Query/Continuous/IContinuousQueryHandle.cs  |   51 +
 .../Cache/Query/IQueryCursor.cs                 |   40 +
 .../Apache.Ignite.Core/Cache/Query/QueryBase.cs |   82 +
 .../Apache.Ignite.Core/Cache/Query/ScanQuery.cs |   77 +
 .../Cache/Query/SqlFieldsQuery.cs               |   81 +
 .../Apache.Ignite.Core/Cache/Query/SqlQuery.cs  |  119 +
 .../Apache.Ignite.Core/Cache/Query/TextQuery.cs |  104 +
 .../Store/CacheParallelLoadStoreAdapter.cs      |  205 ++
 .../Cache/Store/CacheStoreAdapter.cs            |  146 +
 .../Cache/Store/CacheStoreException.cs          |   66 +
 .../Cache/Store/ICacheStore.cs                  |  184 +
 .../Cache/Store/ICacheStoreSession.cs           |   42 +
 .../Cluster/ClusterGroupEmptyException.cs       |   70 +
 .../Cluster/ClusterTopologyException.cs         |   69 +
 .../Apache.Ignite.Core/Cluster/ICluster.cs      |   80 +
 .../Apache.Ignite.Core/Cluster/IClusterGroup.cs |  229 ++
 .../Cluster/IClusterMetrics.cs                  |  515 +++
 .../Apache.Ignite.Core/Cluster/IClusterNode.cs  |  138 +
 .../Cluster/IClusterNodeFilter.cs               |   32 +
 .../Apache.Ignite.Core/Common/IAsyncSupport.cs  |    2 +-
 .../Common/IgniteException.cs                   |    2 +-
 .../Apache.Ignite.Core/Common/IgniteGuid.cs     |  138 +
 .../ComputeExecutionRejectedException.cs        |   69 +
 .../Compute/ComputeJobAdapter.cs                |  122 +
 .../Compute/ComputeJobFailoverException.cs      |   72 +
 .../Compute/ComputeJobResultPolicy.cs           |   45 +
 .../Compute/ComputeTaskAdapter.cs               |   93 +
 .../Compute/ComputeTaskCancelledException.cs    |   69 +
 .../ComputeTaskNoResultCacheAttribute.cs        |   35 +
 .../Compute/ComputeTaskSplitAdapter.cs          |   95 +
 .../Compute/ComputeTaskTimeoutException.cs      |   67 +
 .../Compute/ComputeUserUndeclaredException.cs   |   70 +
 .../Apache.Ignite.Core/Compute/ICompute.cs      |  274 ++
 .../Apache.Ignite.Core/Compute/IComputeFunc.cs  |   55 +
 .../Apache.Ignite.Core/Compute/IComputeJob.cs   |   58 +
 .../Compute/IComputeJobResult.cs                |   73 +
 .../Compute/IComputeReducer.cs                  |   39 +
 .../Apache.Ignite.Core/Compute/IComputeTask.cs  |  132 +
 .../Datastream/IDataStreamer.cs                 |  206 ++
 .../Datastream/IStreamReceiver.cs               |   38 +
 .../Datastream/StreamTransformer.cs             |   73 +
 .../Datastream/StreamVisitor.cs                 |   55 +
 .../Apache.Ignite.Core/Events/CacheEvent.cs     |  176 +
 .../Events/CacheQueryExecutedEvent.cs           |   97 +
 .../Events/CacheQueryReadEvent.cs               |  134 +
 .../Events/CacheRebalancingEvent.cs             |   98 +
 .../Events/CheckpointEvent.cs                   |   50 +
 .../Apache.Ignite.Core/Events/DiscoveryEvent.cs |   80 +
 .../Apache.Ignite.Core/Events/EventBase.cs      |  160 +
 .../Apache.Ignite.Core/Events/EventReader.cs    |   72 +
 .../Apache.Ignite.Core/Events/EventType.cs      |  514 +++
 .../dotnet/Apache.Ignite.Core/Events/IEvent.cs  |   74 +
 .../Apache.Ignite.Core/Events/IEventFilter.cs   |   36 +
 .../dotnet/Apache.Ignite.Core/Events/IEvents.cs |  182 +
 .../Apache.Ignite.Core/Events/JobEvent.cs       |  100 +
 .../Apache.Ignite.Core/Events/SwapSpaceEvent.cs |   50 +
 .../Apache.Ignite.Core/Events/TaskEvent.cs      |   91 +
 .../main/dotnet/Apache.Ignite.Core/IIgnite.cs   |  168 +
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  140 +
 .../main/dotnet/Apache.Ignite.Core/Ignition.cs  |  626 +++-
 .../Impl/Cache/CacheAffinityImpl.cs             |  275 ++
 .../Apache.Ignite.Core/Impl/Cache/CacheEntry.cs |  126 +
 .../Impl/Cache/CacheEntryFilterHolder.cs        |  147 +
 .../Impl/Cache/CacheEntryProcessorHolder.cs     |  145 +
 .../Impl/Cache/CacheEntryProcessorResult.cs     |   65 +
 .../Cache/CacheEntryProcessorResultHolder.cs    |  127 +
 .../Impl/Cache/CacheEnumerable.cs               |   82 +
 .../Impl/Cache/CacheEnumerator.cs               |  117 +
 .../Impl/Cache/CacheEnumeratorProxy.cs          |  156 +
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |  932 +++++
 .../Apache.Ignite.Core/Impl/Cache/CacheLock.cs  |  171 +
 .../Impl/Cache/CacheMetricsImpl.cs              |  248 ++
 .../Apache.Ignite.Core/Impl/Cache/CacheOp.cs    |   63 +
 .../Impl/Cache/CacheProxyImpl.cs                |  499 +++
 .../Impl/Cache/Event/CacheEntryCreateEvent.cs   |   74 +
 .../Impl/Cache/Event/CacheEntryRemoveEvent.cs   |   74 +
 .../Impl/Cache/Event/CacheEntryUpdateEvent.cs   |   79 +
 .../Impl/Cache/MutableCacheEntry.cs             |  163 +
 .../Impl/Cache/Query/AbstractQueryCursor.cs     |  264 ++
 .../Query/Continuous/ContinuousQueryFilter.cs   |  125 +
 .../Continuous/ContinuousQueryFilterHolder.cs   |  118 +
 .../Continuous/ContinuousQueryHandleImpl.cs     |  216 ++
 .../Query/Continuous/ContinuousQueryUtils.cs    |  115 +
 .../Impl/Cache/Query/FieldsQueryCursor.cs       |   54 +
 .../Impl/Cache/Query/QueryCursor.cs             |   50 +
 .../Impl/Cache/Store/CacheStore.cs              |  263 ++
 .../Impl/Cache/Store/CacheStoreSession.cs       |   53 +
 .../Impl/Cache/Store/CacheStoreSessionProxy.cs  |   63 +
 .../Impl/Cluster/ClusterGroupImpl.cs            |  577 +++
 .../Impl/Cluster/ClusterMetricsImpl.cs          |  292 ++
 .../Impl/Cluster/ClusterNodeImpl.cs             |  221 ++
 .../Impl/Cluster/IClusterGroupEx.cs             |   35 +
 .../Impl/Common/DelegateTypeDescriptor.cs       |  327 ++
 .../Apache.Ignite.Core/Impl/Common/Future.cs    |    3 +-
 .../Impl/Common/FutureConverter.cs              |   62 +
 .../Impl/Common/GridArgumentCheck.cs            |   76 -
 .../Impl/Common/IgniteArgumentCheck.cs          |   76 +
 .../Impl/Common/PortableResultWrapper.cs        |   68 +
 .../Closure/ComputeAbstractClosureTask.cs       |  101 +
 .../Impl/Compute/Closure/ComputeActionJob.cs    |   83 +
 .../Impl/Compute/Closure/ComputeFuncJob.cs      |   89 +
 .../Compute/Closure/ComputeMultiClosureTask.cs  |   56 +
 .../Impl/Compute/Closure/ComputeOutFuncJob.cs   |   76 +
 .../Closure/ComputeReducingClosureTask.cs       |   61 +
 .../Compute/Closure/ComputeSingleClosureTask.cs |   48 +
 .../Compute/Closure/IComputeResourceInjector.cs |   31 +
 .../Apache.Ignite.Core/Impl/Compute/Compute.cs  |  213 ++
 .../Impl/Compute/ComputeAsync.cs                |  261 ++
 .../Impl/Compute/ComputeFunc.cs                 |  119 +
 .../Impl/Compute/ComputeImpl.cs                 |  645 ++++
 .../Impl/Compute/ComputeJob.cs                  |  163 +
 .../Impl/Compute/ComputeJobHolder.cs            |  240 ++
 .../Compute/ComputeJobResultGenericWrapper.cs   |   70 +
 .../Impl/Compute/ComputeJobResultImpl.cs        |   96 +
 .../Impl/Compute/ComputeOutFunc.cs              |  123 +
 .../Impl/Compute/ComputeTaskHolder.cs           |  484 +++
 .../Impl/Datastream/DataStreamerBatch.cs        |  269 ++
 .../Impl/Datastream/DataStreamerEntry.cs        |   64 +
 .../Impl/Datastream/DataStreamerImpl.cs         |  832 +++++
 .../Impl/Datastream/DataStreamerRemoveEntry.cs  |   48 +
 .../Impl/Datastream/StreamReceiverHolder.cs     |  144 +
 .../Apache.Ignite.Core/Impl/Events/Events.cs    |  498 +++
 .../Impl/Events/EventsAsync.cs                  |  158 +
 .../Impl/Events/RemoteListenEventFilter.cs      |   85 +
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs   |  204 ++
 .../Impl/Handle/HandleRegistry.cs               |    4 +-
 .../Apache.Ignite.Core/Impl/IInteropCallback.cs |   34 +
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |  547 +++
 .../Impl/IgniteConfigurationEx.cs               |   57 +
 .../Apache.Ignite.Core/Impl/IgniteManager.cs    |  492 +++
 .../Apache.Ignite.Core/Impl/IgniteProxy.cs      |  351 ++
 .../Apache.Ignite.Core/Impl/IgniteUtils.cs      |  438 +++
 .../Impl/Interop/InteropDotNetConfiguration.cs  |   62 +
 .../InteropDotNetPortableConfiguration.cs       |  127 +
 .../InteropDotNetPortableTypeConfiguration.cs   |  151 +
 .../Impl/InteropExceptionHolder.cs              |   85 +
 .../Impl/LifecycleBeanHolder.cs                 |   66 +
 .../Impl/Memory/InteropExternalMemory.cs        |   46 +
 .../Impl/Memory/InteropMemoryUtils.cs           |   38 +
 .../Impl/Memory/PlatformMemoryManager.cs        |    5 +-
 .../Impl/Memory/PlatformMemoryStream.cs         |    3 +-
 .../Impl/Memory/PlatformMemoryUtils.cs          |    2 +-
 .../Impl/Memory/PlatformPooledMemory.cs         |    2 +-
 .../Impl/Messaging/MessageFilterHolder.cs       |  179 +
 .../Impl/Messaging/Messaging.cs                 |  262 ++
 .../Impl/Messaging/MessagingAsync.cs            |   68 +
 .../Apache.Ignite.Core/Impl/NativeMethods.cs    |   47 +
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |  715 ++++
 .../Portable/IPortableSystemTypeSerializer.cs   |   34 +
 .../Impl/Portable/IPortableTypeDescriptor.cs    |  124 +
 .../Impl/Portable/IPortableWriteAware.cs        |   34 +
 .../Impl/Portable/Io/PortableAbstractStream.cs  | 1299 +++++++
 .../Impl/Portable/Io/PortableHeapStream.cs      |  447 +++
 .../Impl/Portable/Io/PortableStreamAdapter.cs   |  114 +
 .../Metadata/IPortableMetadataHandler.cs        |   41 +
 .../Metadata/PortableHashsetMetadataHandler.cs  |   69 +
 .../Portable/Metadata/PortableMetadataHolder.cs |  149 +
 .../Portable/Metadata/PortableMetadataImpl.cs   |  200 ++
 .../Impl/Portable/PortableBuilderField.cs       |   73 +
 .../Impl/Portable/PortableBuilderImpl.cs        |  923 +++++
 .../Impl/Portable/PortableCollectionInfo.cs     |  251 ++
 .../Impl/Portable/PortableFullTypeDescriptor.cs |  203 ++
 .../Impl/Portable/PortableHandleDictionary.cs   |  187 +
 .../Portable/PortableMarshalAwareSerializer.cs  |   45 +
 .../Impl/Portable/PortableMarshaller.cs         |  603 ++++
 .../Impl/Portable/PortableMode.cs               |   40 +
 .../Impl/Portable/PortableObjectHandle.cs       |   59 +
 .../PortableOrSerializableObjectHolder.cs       |   66 +
 .../Portable/PortableReaderHandleDictionary.cs  |   42 +
 .../Impl/Portable/PortableReaderImpl.cs         | 1013 ++++++
 .../Impl/Portable/PortableReflectiveRoutines.cs |  483 +++
 .../Portable/PortableReflectiveSerializer.cs    |  218 ++
 .../Portable/PortableSurrogateTypeDescriptor.cs |  133 +
 .../Impl/Portable/PortableSystemHandlers.cs     | 1336 +++++++
 .../Portable/PortableSystemTypeSerializer.cs    |   62 +
 .../Impl/Portable/PortableUserObject.cs         |  385 ++
 .../Impl/Portable/PortableUtils.cs              | 2039 +++++++++++
 .../Impl/Portable/PortableWriterImpl.cs         | 1305 +++++++
 .../Impl/Portable/PortablesImpl.cs              |  205 ++
 .../Impl/Portable/SerializableObjectHolder.cs   |   66 +
 .../Impl/Portable/TypeResolver.cs               |  227 ++
 .../Impl/Resource/IResourceInjector.cs          |   27 +
 .../Impl/Resource/ResourceFieldInjector.cs      |   47 +
 .../Impl/Resource/ResourceMethodInjector.cs     |   48 +
 .../Impl/Resource/ResourceProcessor.cs          |  105 +
 .../Impl/Resource/ResourcePropertyInjector.cs   |   47 +
 .../Impl/Resource/ResourceTypeDescriptor.cs     |  291 ++
 .../Impl/Services/ServiceContext.cs             |   60 +
 .../Impl/Services/ServiceDescriptor.cs          |  106 +
 .../Impl/Services/ServiceProxy.cs               |   71 +
 .../Impl/Services/ServiceProxyInvoker.cs        |  136 +
 .../Impl/Services/ServiceProxySerializer.cs     |  140 +
 .../Impl/Services/Services.cs                   |  316 ++
 .../Impl/Services/ServicesAsync.cs              |   89 +
 .../Impl/Transactions/AsyncTransaction.cs       |   78 +
 .../Impl/Transactions/Transaction.cs            |  155 +
 .../Impl/Transactions/TransactionImpl.cs        |  489 +++
 .../Impl/Transactions/TransactionMetricsImpl.cs |   62 +
 .../Impl/Transactions/TransactionsImpl.cs       |  201 ++
 .../Impl/Unmanaged/IUnmanagedTarget.cs          |   42 +
 .../Impl/Unmanaged/UnmanagedCallbackHandlers.cs |   99 +
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        | 1152 ++++++
 .../Impl/Unmanaged/UnmanagedContext.cs          |   53 +
 .../Unmanaged/UnmanagedNonReleaseableTarget.cs  |   68 +
 .../Impl/Unmanaged/UnmanagedTarget.cs           |   77 +
 .../Impl/Unmanaged/UnmanagedUtils.cs            | 1252 +++++++
 .../Lifecycle/ILifecycleBean.cs                 |   64 +
 .../Lifecycle/LifecycleEventType.cs             |   49 +
 .../Messaging/IMessageFilter.cs                 |   35 +
 .../Apache.Ignite.Core/Messaging/IMessaging.cs  |  105 +
 .../Portable/IPortableBuilder.cs                |   78 +
 .../Portable/IPortableIdMapper.cs               |   40 +
 .../Portable/IPortableMarshalAware.cs           |   39 +
 .../Portable/IPortableMetadata.cs               |   61 +
 .../Portable/IPortableNameMapper.cs             |   39 +
 .../Portable/IPortableObject.cs                 |   44 +
 .../Portable/IPortableRawReader.cs              |  264 ++
 .../Portable/IPortableRawWriter.cs              |  221 ++
 .../Portable/IPortableReader.cs                 |  340 ++
 .../Portable/IPortableSerializer.cs             |   39 +
 .../Portable/IPortableWriter.cs                 |  259 ++
 .../Apache.Ignite.Core/Portable/IPortables.cs   |  120 +
 .../Portable/PortableConfiguration.cs           |  122 +
 .../Portable/PortableException.cs               |   64 +
 .../Portable/PortableTypeConfiguration.cs       |  162 +
 .../Portable/PortableTypeNames.cs               |  115 +
 .../Resource/InstanceResourceAttribute.cs       |   35 +
 .../Resource/StoreSessionResourceAttribute.cs   |   32 +
 .../Apache.Ignite.Core/Services/IService.cs     |   51 +
 .../Services/IServiceContext.cs                 |   69 +
 .../Services/IServiceDescriptor.cs              |   96 +
 .../Apache.Ignite.Core/Services/IServices.cs    |  181 +
 .../Services/ServiceConfiguration.cs            |   62 +
 .../Services/ServiceInvocationException.cs      |  101 +
 .../Transactions/ITransaction.cs                |  230 ++
 .../Transactions/ITransactionMetrics.cs         |   47 +
 .../Transactions/ITransactions.cs               |   73 +
 .../Transactions/TransactionConcurrency.cs      |   36 +
 .../TransactionHeuristicException.cs            |   72 +
 .../Transactions/TransactionIsolation.cs        |   41 +
 .../TransactionOptimisticException.cs           |   69 +
 .../TransactionRollbackException.cs             |   68 +
 .../Transactions/TransactionState.cs            |   70 +
 .../Transactions/TransactionTimeoutException.cs |   69 +
 .../platform/src/main/dotnet/Apache.Ignite.sln  |   27 +-
 .../ignite/platform/cpp/package-info.java       |   22 +
 .../dotnet/PlatformDotNetConfiguration.java     |  119 -
 .../PlatformDotNetPortableConfiguration.java    |  228 --
 ...PlatformDotNetPortableTypeConfiguration.java |  248 --
 .../ignite/platform/dotnet/package-info.java    |   22 +
 .../apache/ignite/platform/package-info.java    |   22 +
 .../Apache.Ignite.Core.Tests.TestDll.csproj     |   52 +
 .../Properties/AssemblyInfo.cs                  |   49 +
 .../TestClass.cs                                |   35 +
 .../Apache.Ignite.Core.Tests.csproj             |  165 +-
 .../Cache/CacheAbstractTest.cs                  | 3321 ++++++++++++++++++
 .../Cache/CacheAffinityTest.cs                  |  139 +
 .../Cache/CacheDynamicStartTest.cs              |  281 ++
 .../Cache/CacheEntryTest.cs                     |   69 +
 .../Cache/CacheForkedTest.cs                    |   82 +
 .../Cache/CacheLocalAtomicTest.cs               |   57 +
 .../Cache/CacheLocalTest.cs                     |   56 +
 .../CachePartitionedAtomicNearEnabledTest.cs    |   50 +
 .../Cache/CachePartitionedAtomicTest.cs         |   50 +
 .../Cache/CachePartitionedNearEnabledTest.cs    |   50 +
 .../Cache/CachePartitionedTest.cs               |   50 +
 .../Cache/CacheReplicatedAtomicTest.cs          |   60 +
 .../Cache/CacheReplicatedTest.cs                |   60 +
 .../Cache/CacheTestAsyncWrapper.cs              |  436 +++
 .../Cache/Query/CacheQueriesTest.cs             |  928 +++++
 .../Continuous/ContinuousQueryAbstractTest.cs   | 1181 +++++++
 .../ContinuousQueryAtomicBackupTest.cs          |   33 +
 .../ContinuousQueryAtomicNoBackupTest.cs        |   34 +
 .../ContinuousQueryNoBackupAbstractTest.cs      |   72 +
 .../ContinuousQueryTransactionalBackupTest.cs   |   34 +
 .../ContinuousQueryTransactionalNoBackupTest.cs |   33 +
 .../Cache/Store/CacheParallelLoadStoreTest.cs   |  110 +
 .../Cache/Store/CacheStoreSessionTest.cs        |  285 ++
 .../Cache/Store/CacheStoreTest.cs               |  510 +++
 .../Cache/Store/CacheTestParallelLoadStore.cs   |   91 +
 .../Cache/Store/CacheTestStore.cs               |  155 +
 .../Compute/AbstractTaskTest.cs                 |  217 ++
 .../Compute/ClosureTaskTest.cs                  |  390 ++
 .../Compute/ComputeApiTest.cs                   | 1281 +++++++
 .../Compute/ComputeMultithreadedTest.cs         |  269 ++
 .../Compute/FailoverTaskSelfTest.cs             |  246 ++
 .../Forked/ForkedPortableClosureTaskTest.cs     |   33 +
 .../Compute/Forked/ForkedResourceTaskTest.cs    |   33 +
 .../Forked/ForkedSerializableClosureTaskTest.cs |   33 +
 .../Compute/Forked/ForkedTaskAdapterTest.cs     |   33 +
 .../Compute/IgniteExceptionTaskSelfTest.cs      |  753 ++++
 .../Compute/PortableClosureTaskTest.cs          |  217 ++
 .../Compute/PortableTaskTest.cs                 |  253 ++
 .../Compute/ResourceTaskTest.cs                 |  568 +++
 .../Compute/SerializableClosureTaskTest.cs      |  217 ++
 .../Compute/TaskAdapterTest.cs                  |  274 ++
 .../Compute/TaskResultTest.cs                   |  437 +++
 .../Config/Cache/Store/cache-store-session.xml  |   80 +
 .../Config/Compute/compute-grid1.xml            |   90 +
 .../Config/Compute/compute-grid2.xml            |   63 +
 .../Config/Compute/compute-grid3.xml            |   52 +
 .../Config/Compute/compute-standalone.xml       |   87 +
 .../Config/Dynamic/dynamic-client.xml           |   51 +
 .../Config/Dynamic/dynamic-data-no-cfg.xml      |   47 +
 .../Config/Dynamic/dynamic-data.xml             |   65 +
 .../Config/Ignite.exe.config.test               |   41 +
 .../Config/Lifecycle/lifecycle-beans.xml        |   66 +
 .../Config/Lifecycle/lifecycle-no-beans.xml     |   44 +
 .../Config/cache-portables.xml                  |   78 +
 .../Config/cache-query-continuous.xml           |  171 +
 .../Config/cache-query.xml                      |  100 +
 .../Config/marshaller-default.xml               |   43 +
 .../Config/marshaller-invalid.xml               |   46 +
 .../Config/marshaller-portable.xml              |   43 +
 .../native-client-test-cache-affinity.xml       |   70 +
 .../native-client-test-cache-parallel-store.xml |   69 +
 .../native-client-test-cache-portables.xml      |  226 ++
 .../Config/native-client-test-cache-store.xml   |  125 +
 .../Config/native-client-test-cache.xml         |  224 ++
 .../Config/portable.xml                         |   56 +
 .../Config/start-test-grid1.xml                 |   54 +
 .../Config/start-test-grid2.xml                 |   45 +
 .../Config/start-test-grid3.xml                 |   43 +
 .../Dataload/DataStreamerTest.cs                |  592 ++++
 .../Apache.Ignite.Core.Tests/EventsTest.cs      |  961 +++++
 .../Apache.Ignite.Core.Tests/ExceptionsTest.cs  |  352 ++
 .../Apache.Ignite.Core.Tests/ExecutableTest.cs  |  444 +++
 .../Apache.Ignite.Core.Tests/FutureTest.cs      |  278 ++
 .../IgniteManagerTest.cs                        |   51 +
 .../IgniteStartStopTest.cs                      |  380 ++
 .../Apache.Ignite.Core.Tests/IgnitionTest.cs    |   30 -
 .../Apache.Ignite.Core.Tests/LifecycleTest.cs   |  288 ++
 .../Apache.Ignite.Core.Tests/LoadDllTest.cs     |  243 ++
 .../Apache.Ignite.Core.Tests/MarshallerTest.cs  |   71 +
 .../Apache.Ignite.Core.Tests/MessagingTest.cs   |  646 ++++
 .../Portable/PortableApiSelfTest.cs             | 1787 ++++++++++
 .../Portable/PortableSelfTest.cs                | 2078 +++++++++++
 .../PortableConfigurationTest.cs                |  173 +
 .../Process/IIgniteProcessOutputReader.cs       |   35 +
 .../Process/IgniteProcess.cs                    |  292 ++
 .../Process/IgniteProcessConsoleOutputReader.cs |   40 +
 .../Query/ImplicitPortablePerson.cs             |   46 +
 .../Query/NoDefPortablePerson.cs                |   35 +
 .../Query/PortablePerson.cs                     |   69 +
 .../SerializationTest.cs                        |  240 ++
 .../Services/ServiceProxyTest.cs                |  741 ++++
 .../Services/ServicesAsyncWrapper.cs            |  174 +
 .../Services/ServicesTest.cs                    |  823 +++++
 .../Services/ServicesTestAsync.cs               |   33 +
 .../Apache.Ignite.Core.Tests/TestRunner.cs      |   15 +-
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |  292 ++
 .../TypeResolverTest.cs                         |  107 +
 .../platform/PlatformComputeBroadcastTask.java  |   73 +
 .../platform/PlatformComputeDecimalTask.java    |  106 +
 .../platform/PlatformComputeEchoTask.java       |  188 +
 .../ignite/platform/PlatformComputeEnum.java    |   28 +
 .../platform/PlatformComputeJavaPortable.java   |   39 +
 .../platform/PlatformComputePortable.java       |   42 +
 .../PlatformComputePortableArgTask.java         |  119 +
 .../platform/PlatformEventsWriteEventTask.java  |  146 +
 .../ignite/platform/PlatformMaxMemoryTask.java  |   57 +
 .../ignite/platform/PlatformMinMemoryTask.java  |   57 +
 .../lifecycle/PlatformJavaLifecycleBean.java    |   47 +
 .../lifecycle/PlatformJavaLifecycleTask.java    |   65 +
 parent/pom.xml                                  |   18 +
 553 files changed, 102707 insertions(+), 1099 deletions(-)
----------------------------------------------------------------------



[29/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs
new file mode 100644
index 0000000..789e1c4
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs
@@ -0,0 +1,645 @@
+/*
+ * 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.Compute
+{
+    using System;
+    using System.Collections;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Linq;
+    using System.Runtime.Serialization;
+    using System.Threading;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Impl.Cluster;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Compute.Closure;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+    using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils;
+
+    /// <summary>
+    /// Compute implementation.
+    /// </summary>
+    [SuppressMessage("Microsoft.Design", "CA1001:TypesThatOwnDisposableFieldsShouldBeDisposable")]
+    internal class ComputeImpl : PlatformTarget
+    {
+        /** */
+        private const int OpAffinity = 1;
+
+        /** */
+        private const int OpBroadcast = 2;
+
+        /** */
+        private const int OpExec = 3;
+
+        /** */
+        private const int OpExecAsync = 4;
+
+        /** */
+        private const int OpUnicast = 5;
+
+        /** Underlying projection. */
+        private readonly ClusterGroupImpl _prj;
+
+        /** Whether objects must be kept portable. */
+        private readonly ThreadLocal<bool> _keepPortable = new ThreadLocal<bool>(() => false);
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="target">Target.</param>
+        /// <param name="marsh">Marshaller.</param>
+        /// <param name="prj">Projection.</param>
+        /// <param name="keepPortable">"keepPortable" flag.</param>
+        public ComputeImpl(IUnmanagedTarget target, PortableMarshaller marsh, ClusterGroupImpl prj, bool keepPortable)
+            : base(target, marsh)
+        {
+            _prj = prj;
+
+            _keepPortable.Value = keepPortable;
+        }
+
+        /// <summary>
+        /// Grid projection to which this compute instance belongs.
+        /// </summary>
+        public IClusterGroup ClusterGroup
+        {
+            get
+            {
+                return _prj;
+            }
+        }
+
+        /// <summary>
+        /// Sets no-failover flag for the next executed task on this projection in the current thread.
+        /// If flag is set, job will be never failed over even if remote node crashes or rejects execution.
+        /// When task starts execution, the no-failover flag is reset, so all other task will use default
+        /// failover policy, unless this flag is set again.
+        /// </summary>
+        public void WithNoFailover()
+        {
+            UU.ComputeWithNoFailover(Target);
+        }
+
+        /// <summary>
+        /// Sets task timeout for the next executed task on this projection in the current thread.
+        /// When task starts execution, the timeout is reset, so one timeout is used only once.
+        /// </summary>
+        /// <param name="timeout">Computation timeout in milliseconds.</param>
+        public void WithTimeout(long timeout)
+        {
+            UU.ComputeWithTimeout(Target, timeout);
+        }
+
+        /// <summary>
+        /// Sets keep-portable flag for the next executed Java task on this projection in the current
+        /// thread so that task argument passed to Java and returned task results will not be
+        /// deserialized.
+        /// </summary>
+        public void WithKeepPortable()
+        {
+            _keepPortable.Value = true;
+        }
+
+        /// <summary>
+        /// Executes given Java task on the grid projection. If task for given name has not been deployed yet,
+        /// then 'taskName' will be used as task class name to auto-deploy the task.
+        /// </summary>
+        public T ExecuteJavaTask<T>(string taskName, object taskArg)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(taskName, "taskName");
+
+            ICollection<IClusterNode> nodes = _prj.Predicate == null ? null : _prj.Nodes();
+
+            try
+            {
+                T res = DoOutInOp<T>(OpExec, writer =>
+                {
+                    WriteTask(writer, taskName, taskArg, nodes);
+                });
+
+                return res;
+            }
+            finally
+            {
+                _keepPortable.Value = false;
+            }
+        }
+
+        /// <summary>
+        /// Executes given Java task asynchronously on the grid projection.
+        /// If task for given name has not been deployed yet,
+        /// then 'taskName' will be used as task class name to auto-deploy the task.
+        /// </summary>
+        public IFuture<T> ExecuteJavaTaskAsync<T>(string taskName, object taskArg)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(taskName, "taskName");
+
+            ICollection<IClusterNode> nodes = _prj.Predicate == null ? null : _prj.Nodes();
+
+            try
+            {
+                IFuture<T> fut = null;
+
+                DoOutInOp(OpExecAsync, writer =>
+                {
+                    WriteTask(writer, taskName, taskArg, nodes);
+                }, input =>
+                {
+                    fut = GetFuture<T>((futId, futTyp) => UU.TargetListenFuture(Target, futId, futTyp), _keepPortable.Value);
+                });
+
+                return fut;
+            }
+            finally
+            {
+                _keepPortable.Value = false;
+            }
+        }
+
+        /// <summary>
+        /// Executes given task on the grid projection. For step-by-step explanation of task execution process
+        /// refer to <see cref="IComputeTask{A,T,R}"/> documentation.
+        /// </summary>
+        /// <param name="task">Task to execute.</param>
+        /// <param name="taskArg">Optional task argument.</param>
+        /// <returns>Task result.</returns>
+        public IFuture<TR> Execute<TA, T, TR>(IComputeTask<TA, T, TR> task, TA taskArg)
+        {
+            IgniteArgumentCheck.NotNull(task, "task");
+
+            var holder = new ComputeTaskHolder<TA, T, TR>((Ignite) _prj.Ignite, this, task, taskArg);
+
+            long ptr = Marshaller.Ignite.HandleRegistry.Allocate(holder);
+
+            UU.ComputeExecuteNative(Target, ptr, _prj.TopologyVersion);
+
+            return holder.Future;
+        }
+
+        /// <summary>
+        /// Executes given task on the grid projection. For step-by-step explanation of task execution process
+        /// refer to <see cref="IComputeTask{A,T,R}"/> documentation.
+        /// </summary>
+        /// <param name="taskType">Task type.</param>
+        /// <param name="taskArg">Optional task argument.</param>
+        /// <returns>Task result.</returns>
+        public IFuture<TR> Execute<TA, T, TR>(Type taskType, TA taskArg)
+        {
+            IgniteArgumentCheck.NotNull(taskType, "taskType");
+
+            object task = FormatterServices.GetUninitializedObject(taskType);
+
+            var task0 = task as IComputeTask<TA, T, TR>;
+
+            if (task0 == null)
+                throw new IgniteException("Task type doesn't implement IComputeTask: " + taskType.Name);
+
+            return Execute(task0, taskArg);
+        }
+
+        /// <summary>
+        /// Executes provided job on a node in this grid projection. The result of the
+        /// job execution is returned from the result closure.
+        /// </summary>
+        /// <param name="clo">Job to execute.</param>
+        /// <returns>Job result for this execution.</returns>
+        public IFuture<TR> Execute<TR>(IComputeFunc<TR> clo)
+        {
+            IgniteArgumentCheck.NotNull(clo, "clo");
+
+            return ExecuteClosures0(new ComputeSingleClosureTask<object, TR, TR>(),
+                new ComputeOutFuncJob(clo.ToNonGeneric()), null, false);
+        }
+
+        /// <summary>
+        /// Executes provided delegate on a node in this grid projection. The result of the
+        /// job execution is returned from the result closure.
+        /// </summary>
+        /// <param name="func">Func to execute.</param>
+        /// <returns>Job result for this execution.</returns>
+        public IFuture<TR> Execute<TR>(Func<TR> func)
+        {
+            IgniteArgumentCheck.NotNull(func, "func");
+
+            var wrappedFunc = new ComputeOutFuncWrapper(func, () => func());
+
+            return ExecuteClosures0(new ComputeSingleClosureTask<object, TR, TR>(),
+                new ComputeOutFuncJob(wrappedFunc), null, false);
+        }
+
+        /// <summary>
+        /// Executes collection of jobs on nodes within this grid projection.
+        /// </summary>
+        /// <param name="clos">Collection of jobs to execute.</param>
+        /// <returns>Collection of job results for this execution.</returns>
+        public IFuture<ICollection<TR>> Execute<TR>(IEnumerable<IComputeFunc<TR>> clos)
+        {
+            IgniteArgumentCheck.NotNull(clos, "clos");
+
+            ICollection<IComputeJob> jobs = new List<IComputeJob>(GetCountOrZero(clos));
+
+            foreach (IComputeFunc<TR> clo in clos)
+                jobs.Add(new ComputeOutFuncJob(clo.ToNonGeneric()));
+
+            return ExecuteClosures0(new ComputeMultiClosureTask<object, TR, ICollection<TR>>(jobs.Count),
+                null, jobs, false);
+        }
+
+        /// <summary>
+        /// Executes collection of jobs on nodes within this grid projection.
+        /// </summary>
+        /// <param name="clos">Collection of jobs to execute.</param>
+        /// <param name="rdc">Reducer to reduce all job results into one individual return value.</param>
+        /// <returns>Collection of job results for this execution.</returns>
+        public IFuture<TR2> Execute<TR1, TR2>(IEnumerable<IComputeFunc<TR1>> clos, IComputeReducer<TR1, TR2> rdc)
+        {
+            IgniteArgumentCheck.NotNull(clos, "clos");
+
+            ICollection<IComputeJob> jobs = new List<IComputeJob>(GetCountOrZero(clos));
+
+            foreach (var clo in clos)
+                jobs.Add(new ComputeOutFuncJob(clo.ToNonGeneric()));
+
+            return ExecuteClosures0(new ComputeReducingClosureTask<object, TR1, TR2>(rdc), null, jobs, false);
+        }
+
+        /// <summary>
+        /// Broadcasts given job to all nodes in grid projection. Every participating node will return a job result.
+        /// </summary>
+        /// <param name="clo">Job to broadcast to all projection nodes.</param>
+        /// <returns>Collection of results for this execution.</returns>
+        public IFuture<ICollection<TR>> Broadcast<TR>(IComputeFunc<TR> clo)
+        {
+            IgniteArgumentCheck.NotNull(clo, "clo");
+
+            return ExecuteClosures0(new ComputeMultiClosureTask<object, TR, ICollection<TR>>(1),
+                new ComputeOutFuncJob(clo.ToNonGeneric()), null, true);
+        }
+
+        /// <summary>
+        /// Broadcasts given closure job with passed in argument to all nodes in grid projection.
+        /// Every participating node will return a job result.
+        /// </summary>
+        /// <param name="clo">Job to broadcast to all projection nodes.</param>
+        /// <param name="arg">Job closure argument.</param>
+        /// <returns>Collection of results for this execution.</returns>
+        public IFuture<ICollection<TR>> Broadcast<T, TR>(IComputeFunc<T, TR> clo, T arg)
+        {
+            IgniteArgumentCheck.NotNull(clo, "clo");
+
+            return ExecuteClosures0(new ComputeMultiClosureTask<object, TR, ICollection<TR>>(1),
+                new ComputeFuncJob(clo.ToNonGeneric(), arg), null, true);
+        }
+
+        /// <summary>
+        /// Broadcasts given job to all nodes in grid projection.
+        /// </summary>
+        /// <param name="action">Job to broadcast to all projection nodes.</param>
+        public IFuture<object> Broadcast(IComputeAction action)
+        {
+            IgniteArgumentCheck.NotNull(action, "action");
+
+            return ExecuteClosures0(new ComputeSingleClosureTask<object, object, object>(),
+                new ComputeActionJob(action), opId: OpBroadcast);
+        }
+
+        /// <summary>
+        /// Executes provided job on a node in this grid projection.
+        /// </summary>
+        /// <param name="action">Job to execute.</param>
+        public IFuture<object> Run(IComputeAction action)
+        {
+            IgniteArgumentCheck.NotNull(action, "action");
+
+            return ExecuteClosures0(new ComputeSingleClosureTask<object, object, object>(),
+                new ComputeActionJob(action));
+        }
+
+        /// <summary>
+        /// Executes collection of jobs on Ignite nodes within this grid projection.
+        /// </summary>
+        /// <param name="actions">Jobs to execute.</param>
+        public IFuture<object> Run(IEnumerable<IComputeAction> actions)
+        {
+            IgniteArgumentCheck.NotNull(actions, "actions");
+
+            var actions0 = actions as ICollection;
+
+            if (actions0 == null)
+            {
+                var jobs = actions.Select(a => new ComputeActionJob(a)).ToList();
+
+                return ExecuteClosures0(new ComputeSingleClosureTask<object, object, object>(), jobs: jobs,
+                    jobsCount: jobs.Count);
+            }
+            else
+            {
+                var jobs = actions.Select(a => new ComputeActionJob(a));
+
+                return ExecuteClosures0(new ComputeSingleClosureTask<object, object, object>(), jobs: jobs,
+                    jobsCount: actions0.Count);
+            }
+        }
+
+        /// <summary>
+        /// Executes provided closure job on a node in this grid projection.
+        /// </summary>
+        /// <param name="clo">Job to run.</param>
+        /// <param name="arg">Job argument.</param>
+        /// <returns>Job result for this execution.</returns>
+        public IFuture<TR> Apply<T, TR>(IComputeFunc<T, TR> clo, T arg)
+        {
+            IgniteArgumentCheck.NotNull(clo, "clo");
+
+            return ExecuteClosures0(new ComputeSingleClosureTask<T, TR, TR>(),
+                new ComputeFuncJob(clo.ToNonGeneric(), arg), null, false);
+        }
+
+        /// <summary>
+        /// Executes provided closure job on nodes within this grid projection. A new job is executed for
+        /// every argument in the passed in collection. The number of actual job executions will be
+        /// equal to size of the job arguments collection.
+        /// </summary>
+        /// <param name="clo">Job to run.</param>
+        /// <param name="args">Job arguments.</param>
+        /// <returns>Collection of job results.</returns>
+        public IFuture<ICollection<TR>> Apply<T, TR>(IComputeFunc<T, TR> clo, IEnumerable<T> args)
+        {
+            IgniteArgumentCheck.NotNull(clo, "clo");
+
+            IgniteArgumentCheck.NotNull(clo, "clo");
+
+            var jobs = new List<IComputeJob>(GetCountOrZero(args));
+
+            var func = clo.ToNonGeneric();
+            
+            foreach (T arg in args)
+                jobs.Add(new ComputeFuncJob(func, arg));
+
+            return ExecuteClosures0(new ComputeMultiClosureTask<T, TR, ICollection<TR>>(jobs.Count),
+                null, jobs, false);
+        }
+
+        /// <summary>
+        /// Executes provided closure job on nodes within this grid projection. A new job is executed for
+        /// every argument in the passed in collection. The number of actual job executions will be
+        /// equal to size of the job arguments collection. The returned job results will be reduced
+        /// into an individual result by provided reducer.
+        /// </summary>
+        /// <param name="clo">Job to run.</param>
+        /// <param name="args">Job arguments.</param>
+        /// <param name="rdc">Reducer to reduce all job results into one individual return value.</param>
+        /// <returns>Reduced job result for this execution.</returns>
+        public IFuture<TR2> Apply<T, TR1, TR2>(IComputeFunc<T, TR1> clo, IEnumerable<T> args,
+            IComputeReducer<TR1, TR2> rdc)
+        {
+            IgniteArgumentCheck.NotNull(clo, "clo");
+
+            IgniteArgumentCheck.NotNull(clo, "clo");
+
+            IgniteArgumentCheck.NotNull(clo, "clo");
+
+            ICollection<IComputeJob> jobs = new List<IComputeJob>(GetCountOrZero(args));
+
+            var func = clo.ToNonGeneric();
+
+            foreach (T arg in args)
+                jobs.Add(new ComputeFuncJob(func, arg));
+
+            return ExecuteClosures0(new ComputeReducingClosureTask<T, TR1, TR2>(rdc),
+                null, jobs, false);
+        }
+
+        /// <summary>
+        /// Executes given job on the node where data for provided affinity key is located
+        /// (a.k.a. affinity co-location).
+        /// </summary>
+        /// <param name="cacheName">Name of the cache to use for affinity co-location.</param>
+        /// <param name="affinityKey">Affinity key.</param>
+        /// <param name="action">Job to execute.</param>
+        public IFuture AffinityRun(string cacheName, object affinityKey, IComputeAction action)
+        {
+            IgniteArgumentCheck.NotNull(action, "action");
+
+            return ExecuteClosures0(new ComputeSingleClosureTask<object, object, object>(),
+                new ComputeActionJob(action), opId: OpAffinity,
+                writeAction: w => WriteAffinity(w, cacheName, affinityKey));
+        }
+
+        /// <summary>
+        /// Executes given job on the node where data for provided affinity key is located
+        /// (a.k.a. affinity co-location).
+        /// </summary>
+        /// <param name="cacheName">Name of the cache to use for affinity co-location.</param>
+        /// <param name="affinityKey">Affinity key.</param>
+        /// <param name="clo">Job to execute.</param>
+        /// <returns>Job result for this execution.</returns>
+        /// <typeparam name="TR">Type of job result.</typeparam>
+        public IFuture<TR> AffinityCall<TR>(string cacheName, object affinityKey, IComputeFunc<TR> clo)
+        {
+            IgniteArgumentCheck.NotNull(clo, "clo");
+
+            return ExecuteClosures0(new ComputeSingleClosureTask<object, TR, TR>(),
+                new ComputeOutFuncJob(clo.ToNonGeneric()), opId: OpAffinity,
+                writeAction: w => WriteAffinity(w, cacheName, affinityKey));
+        }
+
+        /** <inheritDoc /> */
+        protected override T Unmarshal<T>(IPortableStream stream)
+        {
+            bool keep = _keepPortable.Value;
+
+            return Marshaller.Unmarshal<T>(stream, keep);
+        }
+
+        /// <summary>
+        /// Internal routine for closure-based task execution.
+        /// </summary>
+        /// <param name="task">Task.</param>
+        /// <param name="job">Job.</param>
+        /// <param name="jobs">Jobs.</param>
+        /// <param name="broadcast">Broadcast flag.</param>
+        /// <returns>Future.</returns>
+        private IFuture<TR> ExecuteClosures0<TA, T, TR>(IComputeTask<TA, T, TR> task, IComputeJob job,
+            ICollection<IComputeJob> jobs, bool broadcast)
+        {
+            return ExecuteClosures0(task, job, jobs, broadcast ? OpBroadcast : OpUnicast,
+                jobs == null ? 1 : jobs.Count);
+        }
+
+        /// <summary>
+        /// Internal routine for closure-based task execution.
+        /// </summary>
+        /// <param name="task">Task.</param>
+        /// <param name="job">Job.</param>
+        /// <param name="jobs">Jobs.</param>
+        /// <param name="opId">Op code.</param>
+        /// <param name="jobsCount">Jobs count.</param>
+        /// <param name="writeAction">Custom write action.</param>
+        /// <returns>Future.</returns>
+        [SuppressMessage("Microsoft.Design", "CA1031:DoNotCatchGeneralExceptionTypes",
+            Justification = "User code can throw any exception")]
+        private IFuture<TR> ExecuteClosures0<TA, T, TR>(IComputeTask<TA, T, TR> task, IComputeJob job = null,
+            IEnumerable<IComputeJob> jobs = null, int opId = OpUnicast, int jobsCount = 0,
+            Action<PortableWriterImpl> writeAction = null)
+        {
+            Debug.Assert(job != null || jobs != null);
+
+            var holder = new ComputeTaskHolder<TA, T, TR>((Ignite) _prj.Ignite, this, task, default(TA));
+
+            var taskHandle = Marshaller.Ignite.HandleRegistry.Allocate(holder);
+
+            var jobHandles = new List<long>(job != null ? 1 : jobsCount);
+
+            try
+            {
+                Exception err = null;
+
+                try
+                {
+                    DoOutOp(opId, writer =>
+                    {
+                        writer.WriteLong(taskHandle);
+
+                        if (job != null)
+                        {
+                            writer.WriteInt(1);
+
+                            jobHandles.Add(WriteJob(job, writer));
+                        }
+                        else
+                        {
+                            writer.WriteInt(jobsCount);
+
+                            Debug.Assert(jobs != null, "jobs != null");
+
+                            jobHandles.AddRange(jobs.Select(jobEntry => WriteJob(jobEntry, writer)));
+                        }
+                        
+                        holder.JobHandles(jobHandles);
+
+                        if (writeAction != null)
+                            writeAction(writer);
+                    });
+                }
+                catch (Exception e)
+                {
+                    err = e;
+                }
+
+                if (err != null)
+                {
+                    // Manual job handles release because they were not assigned to the task yet.
+                    foreach (var hnd in jobHandles) 
+                        Marshaller.Ignite.HandleRegistry.Release(hnd);
+
+                    holder.CompleteWithError(taskHandle, err);
+                }
+            }
+            catch (Exception e)
+            {
+                // This exception means that out-op failed.
+                holder.CompleteWithError(taskHandle, e);
+            }
+
+            return holder.Future;
+        }
+
+        /// <summary>
+        /// Writes the job.
+        /// </summary>
+        /// <param name="job">The job.</param>
+        /// <param name="writer">The writer.</param>
+        /// <returns>Handle to the job holder</returns>
+        private long WriteJob(IComputeJob job, PortableWriterImpl writer)
+        {
+            var jobHolder = new ComputeJobHolder(_prj.Ignite as Ignite, job);
+
+            var jobHandle = Marshaller.Ignite.HandleRegistry.Allocate(jobHolder);
+
+            writer.WriteLong(jobHandle);
+            writer.WriteObject(jobHolder);
+
+            return jobHandle;
+        }
+
+        /// <summary>
+        /// Write task to the writer.
+        /// </summary>
+        /// <param name="writer">Writer.</param>
+        /// <param name="taskName">Task name.</param>
+        /// <param name="taskArg">Task arg.</param>
+        /// <param name="nodes">Nodes.</param>
+        private void WriteTask(PortableWriterImpl writer, string taskName, object taskArg,
+            ICollection<IClusterNode> nodes)
+        {
+            writer.WriteString(taskName);
+            writer.WriteBoolean(_keepPortable.Value);
+            writer.Write(taskArg);
+
+            WriteNodeIds(writer, nodes);
+        }
+
+        /// <summary>
+        /// Write node IDs.
+        /// </summary>
+        /// <param name="writer">Writer.</param>
+        /// <param name="nodes">Nodes.</param>
+        private static void WriteNodeIds(PortableWriterImpl writer, ICollection<IClusterNode> nodes)
+        {
+            if (nodes == null)
+                writer.WriteBoolean(false);
+            else
+            {
+                writer.WriteBoolean(true);
+                writer.WriteInt(nodes.Count);
+
+                foreach (IClusterNode node in nodes)
+                    writer.WriteGuid(node.Id);
+            }
+        }
+
+        /// <summary>
+        /// Writes the affinity info.
+        /// </summary>
+        /// <param name="writer">The writer.</param>
+        /// <param name="cacheName">Name of the cache to use for affinity co-location.</param>
+        /// <param name="affinityKey">Affinity key.</param>
+        private static void WriteAffinity(PortableWriterImpl writer, string cacheName, object affinityKey)
+        {
+            writer.WriteString(cacheName);
+
+            writer.WriteObject(affinityKey);
+        }
+
+        /// <summary>
+        /// Gets element count or zero.
+        /// </summary>
+        private static int GetCountOrZero(object collection)
+        {
+            var coll = collection as ICollection;
+
+            return coll == null ? 0 : coll.Count;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJob.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJob.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJob.cs
new file mode 100644
index 0000000..f4ed999
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJob.cs
@@ -0,0 +1,163 @@
+/*
+ * 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.Compute
+{
+    using System;
+    using System.Reflection;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Resource;
+    using Apache.Ignite.Core.Portable;
+    using Apache.Ignite.Core.Resource;
+
+    /// <summary>
+    /// Non-generic version of IComputeJob{T}.
+    /// </summary>
+    internal interface IComputeJob : IComputeJob<object>
+    {
+        // No-op.
+    }
+
+    /// <summary>
+    /// Wraps generic func into a non-generic for internal usage.
+    /// </summary>
+    internal class ComputeJobWrapper : IComputeJob, IPortableWriteAware
+    {
+        /** */
+        private readonly Func<object, object> _execute;
+
+        /** */
+        private readonly Action<object> _cancel;
+
+        /** */
+        private readonly object _job;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeJobWrapper"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        public ComputeJobWrapper(IPortableReader reader)
+        {
+            var reader0 = (PortableReaderImpl)reader.RawReader();
+
+            _job = PortableUtils.ReadPortableOrSerializable<object>(reader0);
+
+            DelegateTypeDescriptor.GetComputeJob(_job.GetType(), out _execute, out _cancel);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeFuncWrapper" /> class.
+        /// </summary>
+        public ComputeJobWrapper(object job, Func<object, object> execute, Action<object> cancel)
+        {
+            _job = job;
+
+            _execute = execute;
+
+            _cancel = cancel;
+        }
+
+        /** <inheritDoc /> */
+        public object Execute()
+        {
+            try
+            {
+                return _execute(_job);
+            }
+            catch (TargetInvocationException ex)
+            {
+                throw ex.InnerException;
+            }
+        }
+
+        /** <inheritDoc /> */
+        public void Cancel()
+        {
+            try
+            {
+                _cancel(_job);
+            }
+            catch (TargetInvocationException ex)
+            {
+                throw ex.InnerException;
+            }
+        }
+
+        /** <inheritDoc /> */
+        public void WritePortable(IPortableWriter writer)
+        {
+            var writer0 = (PortableWriterImpl)writer.RawWriter();
+
+            writer0.DetachNext();
+            PortableUtils.WritePortableOrSerializable(writer0, Job);
+        }
+
+        /// <summary>
+        /// Injects Ignite instance into wrapped object.
+        /// </summary>
+        [InstanceResource]
+        public void InjectIgnite(IIgnite ignite)
+        {
+            // Propagate injection
+            ResourceProcessor.Inject(Job, (IgniteProxy)ignite);
+        }
+
+        /// <summary>
+        /// Gets the inner job.
+        /// </summary>
+        public object Job
+        {
+            get { return _job; }
+        }
+    }
+
+    /// <summary>
+    /// Extension methods for IComputeJob{T}.
+    /// </summary>
+    internal static class ComputeJobExtensions
+    {
+        /// <summary>
+        /// Convert to non-generic wrapper.
+        /// </summary>
+        public static IComputeJob ToNonGeneric<T>(this IComputeJob<T> job)
+        {
+            return new ComputeJobWrapper(job, x => job.Execute(), x => job.Cancel());
+        }
+
+        /// <summary>
+        /// Unwraps job of one type into job of another type.
+        /// </summary>
+        public static IComputeJob<TR> Unwrap<T, TR>(this IComputeJob<T> job)
+        {
+            var wrapper = job as ComputeJobWrapper;
+
+            return wrapper != null ? (IComputeJob<TR>) wrapper.Job : (IComputeJob<TR>) job;
+        }
+        
+        /// <summary>
+        /// Unwraps job of one type into job of another type.
+        /// </summary>
+        public static object Unwrap(this IComputeJob<object> job)
+        {
+            var wrapper = job as ComputeJobWrapper;
+
+            return wrapper != null ? wrapper.Job : job;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobHolder.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobHolder.cs
new file mode 100644
index 0000000..9bdb5cf
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobHolder.cs
@@ -0,0 +1,240 @@
+/*
+ * 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.Compute
+{
+    using System;
+    using System.Diagnostics.CodeAnalysis;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Cluster;
+    using Apache.Ignite.Core.Impl.Compute.Closure;
+    using Apache.Ignite.Core.Impl.Memory;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Impl.Resource;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Holder for user-provided compute job.
+    /// </summary>
+    internal class ComputeJobHolder : IPortableWriteAware
+    {
+        /** Actual job. */
+        private readonly IComputeJob _job;
+        
+        /** Owning grid. */
+        private readonly Ignite _ignite;
+
+        /** Result (set for local jobs only). */
+        private volatile ComputeJobResultImpl _jobRes;
+
+        /// <summary>
+        /// Default ctor for marshalling.
+        /// </summary>
+        /// <param name="reader"></param>
+        public ComputeJobHolder(IPortableReader reader)
+        {
+            var reader0 = (PortableReaderImpl) reader.RawReader();
+
+            _ignite = reader0.Marshaller.Ignite;
+
+            _job = PortableUtils.ReadPortableOrSerializable<IComputeJob>(reader0);
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="grid">Grid.</param>
+        /// <param name="job">Job.</param>
+        public ComputeJobHolder(Ignite grid, IComputeJob job)
+        {
+            _ignite = grid;
+            _job = job;
+        }
+
+        /// <summary>
+        /// Executes local job.
+        /// </summary>
+        /// <param name="cancel">Cancel flag.</param>
+        public void ExecuteLocal(bool cancel)
+        {
+            object res;
+            bool success;
+
+            Execute0(cancel, out res, out success);
+
+            _jobRes = new ComputeJobResultImpl(
+                success ? res : null, 
+                success ? null : res as Exception, 
+                _job, 
+                _ignite.LocalNode.Id, 
+                cancel
+            );
+        }
+
+        /// <summary>
+        /// Execute job serializing result to the stream.
+        /// </summary>
+        /// <param name="cancel">Whether the job must be cancelled.</param>
+        /// <param name="stream">Stream.</param>
+        public void ExecuteRemote(PlatformMemoryStream stream, bool cancel)
+        {
+            // 1. Execute job.
+            object res;
+            bool success;
+
+            Execute0(cancel, out res, out success);
+
+            // 2. Try writing result to the stream.
+            ClusterGroupImpl prj = _ignite.ClusterGroup;
+
+            PortableWriterImpl writer = prj.Marshaller.StartMarshal(stream);
+
+            try
+            {
+                // 3. Marshal results.
+                PortableUtils.WriteWrappedInvocationResult(writer, success, res);
+            }
+            finally
+            {
+                // 4. Process metadata.
+                prj.FinishMarshal(writer);
+            }
+        }
+
+        /// <summary>
+        /// Cancel the job.
+        /// </summary>
+        public void Cancel()
+        {
+            _job.Cancel();
+        }
+
+        /// <summary>
+        /// Serialize the job to the stream.
+        /// </summary>
+        /// <param name="stream">Stream.</param>
+        /// <returns>True if successfull.</returns>
+        [SuppressMessage("Microsoft.Design", "CA1031:DoNotCatchGeneralExceptionTypes",
+            Justification = "User job can throw any exception")]
+        internal bool Serialize(IPortableStream stream)
+        {
+            ClusterGroupImpl prj = _ignite.ClusterGroup;
+
+            PortableWriterImpl writer = prj.Marshaller.StartMarshal(stream);
+
+            try
+            {
+                writer.Write(this);
+
+                return true;
+            }
+            catch (Exception e)
+            {
+                writer.WriteString("Failed to marshal job [job=" + _job + ", errType=" + e.GetType().Name +
+                    ", errMsg=" + e.Message + ']');
+
+                return false;
+            }
+            finally
+            {
+                // 4. Process metadata.
+                prj.FinishMarshal(writer);
+            }
+        }
+
+        /// <summary>
+        /// Job.
+        /// </summary>
+        internal IComputeJob Job
+        {
+            get { return _job; }
+        }
+
+        /// <summary>
+        /// Job result.
+        /// </summary>
+        internal ComputeJobResultImpl JobResult
+        {
+            get { return _jobRes; }
+        }
+
+        /// <summary>
+        /// Internal job execution routine.
+        /// </summary>
+        /// <param name="cancel">Cancel flag.</param>
+        /// <param name="res">Result.</param>
+        /// <param name="success">Success flag.</param>
+        [SuppressMessage("Microsoft.Design", "CA1031:DoNotCatchGeneralExceptionTypes",
+            Justification = "User job can throw any exception")]
+        private void Execute0(bool cancel, out object res, out bool success)
+        {
+            // 1. Inject resources.
+            IComputeResourceInjector injector = _job as IComputeResourceInjector;
+
+            if (injector != null)
+                injector.Inject(_ignite);
+            else
+                ResourceProcessor.Inject(_job, _ignite);
+
+            // 2. Execute.
+            try
+            {
+                if (cancel)
+                    _job.Cancel();
+
+                res = _job.Execute();
+
+                success = true;
+            }
+            catch (Exception e)
+            {
+                res = e;
+
+                success = false;
+            }
+        }
+
+        /** <inheritDoc /> */
+        public void WritePortable(IPortableWriter writer)
+        {
+            PortableWriterImpl writer0 = (PortableWriterImpl) writer.RawWriter();
+
+            writer0.DetachNext();
+            PortableUtils.WritePortableOrSerializable(writer0, _job);
+        }
+
+        /// <summary>
+        /// Create job instance.
+        /// </summary>
+        /// <param name="grid">Grid.</param>
+        /// <param name="stream">Stream.</param>
+        /// <returns></returns>
+        internal static ComputeJobHolder CreateJob(Ignite grid, IPortableStream stream)
+        {
+            try
+            {
+                return grid.Marshaller.StartUnmarshal(stream).ReadObject<ComputeJobHolder>();
+            }
+            catch (Exception e)
+            {
+                throw new IgniteException("Failed to deserialize the job [errType=" + e.GetType().Name +
+                    ", errMsg=" + e.Message + ']');
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobResultGenericWrapper.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobResultGenericWrapper.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobResultGenericWrapper.cs
new file mode 100644
index 0000000..8173f71
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobResultGenericWrapper.cs
@@ -0,0 +1,70 @@
+/*
+ * 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.Compute
+{
+    using System;
+    using Apache.Ignite.Core.Compute;
+
+    /// <summary>
+    /// Wraps non-generic IComputeJobResult in generic form.
+    /// </summary>
+    internal class ComputeJobResultGenericWrapper<T> : IComputeJobResult<T>
+    {
+        /** */
+        private readonly IComputeJobResult<object> _wrappedRes;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeJobResultGenericWrapper{T}"/> class.
+        /// </summary>
+        /// <param name="jobRes">The job result to wrap.</param>
+        public ComputeJobResultGenericWrapper(IComputeJobResult<object> jobRes)
+        {
+            _wrappedRes = jobRes;
+        }
+
+        /** <inheritdoc /> */
+        public T Data()
+        {
+            return (T)_wrappedRes.Data();
+        }
+
+        /** <inheritdoc /> */
+        public Exception Exception()
+        {
+            return _wrappedRes.Exception();
+        }
+
+        /** <inheritdoc /> */
+        public IComputeJob<T> Job()
+        {
+            return _wrappedRes.Job().Unwrap<object, T>();
+        }
+
+        /** <inheritdoc /> */
+        public Guid NodeId
+        {
+            get { return _wrappedRes.NodeId; }
+        }
+
+        /** <inheritdoc /> */
+        public bool Cancelled
+        {
+            get { return _wrappedRes.Cancelled; }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobResultImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobResultImpl.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobResultImpl.cs
new file mode 100644
index 0000000..a35bae0
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobResultImpl.cs
@@ -0,0 +1,96 @@
+/*
+ * 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.Compute
+{
+    using System;
+    using Apache.Ignite.Core.Compute;
+
+    /// <summary>
+    /// Job result implementation.
+    /// </summary>
+    internal class ComputeJobResultImpl : IComputeJobResult<object>
+    {
+        /** Data. */
+        private readonly object _data;
+
+        /** Exception. */
+        private readonly Exception _err;
+
+        /** Backing job. */
+        private readonly IComputeJob _job;
+
+        /** Node ID. */
+        private readonly Guid _nodeId;
+
+        /** Cancel flag. */
+        private readonly bool _cancelled;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="data">Data.</param>
+        /// <param name="err">Exception.</param>
+        /// <param name="job">Backing job.</param>
+        /// <param name="nodeId">Node ID.</param>
+        /// <param name="cancelled">Cancel flag.</param>
+        public ComputeJobResultImpl(object data, Exception err, IComputeJob job, Guid nodeId, bool cancelled)
+        {
+            _data = data;
+            _err = err;
+            _job = job;
+            _nodeId = nodeId;
+            _cancelled = cancelled;
+        }
+
+        /** <inheritDoc /> */
+        public object Data()
+        {
+            return _data;
+        }
+
+        /** <inheritDoc /> */
+        public Exception Exception()
+        {
+            return _err;
+        }
+
+        /** <inheritDoc /> */
+        public IComputeJob<object> Job()
+        {
+            return _job;
+        }
+
+        /** <inheritDoc /> */
+        public Guid NodeId
+        {
+            get
+            {
+                return _nodeId;
+            }
+        }
+
+        /** <inheritDoc /> */
+        public bool Cancelled
+        {
+            get 
+            { 
+                return _cancelled; 
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeOutFunc.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeOutFunc.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeOutFunc.cs
new file mode 100644
index 0000000..dda04b6
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeOutFunc.cs
@@ -0,0 +1,123 @@
+/*
+ * 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.Compute
+{
+    using System;
+    using System.Diagnostics;
+    using System.Reflection;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Resource;
+    using Apache.Ignite.Core.Portable;
+    using Apache.Ignite.Core.Resource;
+
+    /// <summary>
+    /// Non-generic version of IComputeFunc{T}.
+    /// </summary>
+    internal interface IComputeOutFunc : IComputeFunc<object>
+    {
+        // No-op.
+    }
+
+    /// <summary>
+    /// Wraps generic func into a non-generic for internal usage.
+    /// </summary>
+    internal class ComputeOutFuncWrapper : IComputeOutFunc, IPortableWriteAware
+    {
+        /** */
+        private readonly object _func;
+
+        /** */
+        private readonly Func<object, object> _invoker;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeFuncWrapper" /> class.
+        /// </summary>
+        /// <param name="func">The function to wrap.</param>
+        /// <param name="invoker">The function invoker.</param>
+        public ComputeOutFuncWrapper(object func, Func<object> invoker)
+        {
+            Debug.Assert(func != null);
+            Debug.Assert(invoker != null);
+
+            _func = func;
+
+            _invoker = target => invoker();
+        }
+
+        /** <inheritDoc /> */
+        public object Invoke()
+        {
+            try
+            {
+                return _invoker(_func);
+            }
+            catch (TargetInvocationException ex)
+            {
+                throw ex.InnerException;
+            }
+        }
+
+        /** <inheritDoc /> */
+        public void WritePortable(IPortableWriter writer)
+        {
+            var writer0 = (PortableWriterImpl)writer.RawWriter();
+
+            writer0.DetachNext();
+            PortableUtils.WritePortableOrSerializable(writer0, _func);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ComputeOutFuncWrapper"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        public ComputeOutFuncWrapper(IPortableReader reader)
+        {
+            var reader0 = (PortableReaderImpl)reader.RawReader();
+
+            _func = PortableUtils.ReadPortableOrSerializable<object>(reader0);
+
+            _invoker = DelegateTypeDescriptor.GetComputeOutFunc(_func.GetType());
+        }
+
+        /// <summary>
+        /// Injects the grid.
+        /// </summary>
+        [InstanceResource]
+        public void InjectIgnite(IIgnite ignite)
+        {
+            // Propagate injection
+            ResourceProcessor.Inject(_func, (IgniteProxy)ignite);
+        }
+    }
+
+    /// <summary>
+    /// Extension methods for IComputeOutFunc{T}.
+    /// </summary>
+    internal static class ComputeOutFuncExtensions
+    {
+        /// <summary>
+        /// Convert to non-generic wrapper.
+        /// </summary>
+        public static IComputeOutFunc ToNonGeneric<T>(this IComputeFunc<T> func)
+        {
+            return new ComputeOutFuncWrapper(func, () => func.Invoke());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeTaskHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeTaskHolder.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeTaskHolder.cs
new file mode 100644
index 0000000..dfe0d18
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeTaskHolder.cs
@@ -0,0 +1,484 @@
+/*
+ * 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.Compute
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Collections.ObjectModel;
+    using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Linq;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Impl.Cluster;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Compute.Closure;
+    using Apache.Ignite.Core.Impl.Memory;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Resource;
+
+    /// <summary>
+    /// Compute task holder interface used to avoid generics.
+    /// </summary>
+    internal interface IComputeTaskHolder
+    {
+        /// <summary>
+        /// Perform map step.
+        /// </summary>
+        /// <param name="inStream">Stream with IN data (topology info).</param>
+        /// <param name="outStream">Stream for OUT data (map result).</param>
+        /// <returns>Map with produced jobs.</returns>
+        void Map(PlatformMemoryStream inStream, PlatformMemoryStream outStream);
+
+        /// <summary>
+        /// Process local job result.
+        /// </summary>
+        /// <param name="jobId">Job pointer.</param>
+        /// <returns>Policy.</returns>
+        int JobResultLocal(ComputeJobHolder jobId);
+
+        /// <summary>
+        /// Process remote job result.
+        /// </summary>
+        /// <param name="jobId">Job pointer.</param>
+        /// <param name="stream">Stream.</param>
+        /// <returns>Policy.</returns>
+        int JobResultRemote(ComputeJobHolder jobId, PlatformMemoryStream stream);
+        
+        /// <summary>
+        /// Perform task reduce.
+        /// </summary>
+        void Reduce();
+
+        /// <summary>
+        /// Complete task.
+        /// </summary>
+        /// <param name="taskHandle">Task handle.</param>
+        void Complete(long taskHandle);
+        
+        /// <summary>
+        /// Complete task with error.
+        /// </summary>
+        /// <param name="taskHandle">Task handle.</param>
+        /// <param name="stream">Stream with serialized exception.</param>
+        void CompleteWithError(long taskHandle, PlatformMemoryStream stream);
+    }
+
+    /// <summary>
+    /// Compute task holder.
+    /// </summary>
+    internal class ComputeTaskHolder<TA, T, TR> : IComputeTaskHolder
+    {
+        /** Empty results. */
+        private static readonly IList<IComputeJobResult<T>> EmptyRes =     
+            new ReadOnlyCollection<IComputeJobResult<T>>(new List<IComputeJobResult<T>>());
+
+        /** Compute instance. */
+        private readonly ComputeImpl _compute;
+
+        /** Actual task. */
+        private readonly IComputeTask<TA, T, TR> _task;
+
+        /** Task argument. */
+        private readonly TA _arg;
+
+        /** Results cache flag. */
+        private readonly bool _resCache;
+
+        /** Task future. */
+        private readonly Future<TR> _fut = new Future<TR>();
+                
+        /** Jobs whose results are cached. */
+        private ISet<object> _resJobs;
+
+        /** Cached results. */
+        private IList<IComputeJobResult<T>> _ress;
+
+        /** Handles for jobs which are not serialized right away. */
+        private volatile List<long> _jobHandles;
+        
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="grid">Grid.</param>
+        /// <param name="compute">Compute.</param>
+        /// <param name="task">Task.</param>
+        /// <param name="arg">Argument.</param>
+        public ComputeTaskHolder(Ignite grid, ComputeImpl compute, IComputeTask<TA, T, TR> task, TA arg)
+        {
+            _compute = compute;
+            _arg = arg;
+            _task = task;
+
+            ResourceTypeDescriptor resDesc = ResourceProcessor.Descriptor(task.GetType());
+
+            IComputeResourceInjector injector = task as IComputeResourceInjector;
+
+            if (injector != null)
+                injector.Inject(grid);
+            else
+                resDesc.InjectIgnite(task, grid);
+
+            _resCache = !resDesc.TaskNoResultCache;
+        }
+
+        /** <inheritDoc /> */
+        [SuppressMessage("Microsoft.Design", "CA1031:DoNotCatchGeneralExceptionTypes",
+            Justification = "User code can throw any exception")]
+        public void Map(PlatformMemoryStream inStream, PlatformMemoryStream outStream)
+        {
+            IList<IClusterNode> subgrid;
+
+            ClusterGroupImpl prj = (ClusterGroupImpl)_compute.ClusterGroup;
+
+            var ignite = (Ignite) prj.Ignite;
+
+            // 1. Unmarshal topology info if topology changed.
+            var reader = prj.Marshaller.StartUnmarshal(inStream);
+
+            if (reader.ReadBoolean())
+            {
+                long topVer = reader.ReadLong();
+
+                List<IClusterNode> nodes = new List<IClusterNode>(reader.ReadInt());
+
+                int nodesCnt = reader.ReadInt();
+
+                subgrid = new List<IClusterNode>(nodesCnt);
+
+                for (int i = 0; i < nodesCnt; i++)
+                {
+                    IClusterNode node = ignite.GetNode(reader.ReadGuid());
+
+                    nodes.Add(node);
+
+                    if (reader.ReadBoolean())
+                        subgrid.Add(node);
+                }
+
+                // Update parent projection to help other task callers avoid this overhead.
+                // Note that there is a chance that topology changed even further and this update fails.
+                // It means that some of subgrid nodes could have left the Grid. This is not critical
+                // for us, because Java will handle it gracefully.
+                prj.UpdateTopology(topVer, nodes);
+            }
+            else
+            {
+                IList<IClusterNode> nodes = prj.NodesNoRefresh();
+
+                Debug.Assert(nodes != null, "At least one topology update should have occurred.");
+
+                subgrid = IgniteUtils.Shuffle(nodes);
+            }
+
+            // 2. Perform map.
+            IDictionary<IComputeJob<T>, IClusterNode> map;
+            Exception err;
+
+            try
+            {
+                map = _task.Map(subgrid, _arg);
+
+                err = null;
+            }
+            catch (Exception e)
+            {
+                map = null;
+
+                err = e;
+
+                // Java can receive another exception in case of marshalling failure but it is not important.
+                Finish(default(TR), e);
+            }
+
+            // 3. Write map result to the output stream.
+            PortableWriterImpl writer = prj.Marshaller.StartMarshal(outStream);
+
+            try
+            {
+                if (err == null)
+                {
+                    writer.WriteBoolean(true); // Success flag.
+
+                    if (map == null)
+                        writer.WriteBoolean(false); // Map produced no result.
+                    else
+                    {
+                        writer.WriteBoolean(true); // Map produced result.
+                        writer.WriteInt(map.Count); // Amount of mapped jobs.
+
+                        var jobHandles = new List<long>(map.Count);
+
+                        foreach (KeyValuePair<IComputeJob<T>, IClusterNode> mapEntry in map)
+                        {
+                            var job = new ComputeJobHolder(_compute.ClusterGroup.Ignite as Ignite, mapEntry.Key.ToNonGeneric());
+
+                            IClusterNode node = mapEntry.Value;
+
+                            var jobHandle = ignite.HandleRegistry.Allocate(job);
+
+                            jobHandles.Add(jobHandle);
+
+                            writer.WriteLong(jobHandle);
+
+                            if (node.IsLocal)
+                                writer.WriteBoolean(false); // Job is not serialized.
+                            else
+                            {
+                                writer.WriteBoolean(true); // Job is serialized.
+                                writer.WriteObject(job);
+                            }
+
+                            writer.WriteGuid(node.Id);
+                        }
+
+                        _jobHandles = jobHandles;
+                    }
+                }
+                else
+                {
+                    writer.WriteBoolean(false); // Map failed.
+
+                    // Write error as string because it is not important for Java, we need only to print
+                    // a message in the log.
+                    writer.WriteString("Map step failed [errType=" + err.GetType().Name +
+                        ", errMsg=" + err.Message + ']');
+                }
+            }
+            catch (Exception e)
+            {
+                // Something went wrong during marshaling.
+                Finish(default(TR), e);
+
+                outStream.Reset();
+                
+                writer.WriteBoolean(false); // Map failed.
+                writer.WriteString(e.Message); // Write error message.
+            }
+            finally
+            {
+                prj.Marshaller.FinishMarshal(writer);
+            }
+        }
+
+        /** <inheritDoc /> */
+        public int JobResultLocal(ComputeJobHolder job)
+        {
+            return (int)JobResult0(job.JobResult);
+        }
+
+        /** <inheritDoc /> */
+        [SuppressMessage("ReSharper", "PossibleInvalidOperationException")]
+        public int JobResultRemote(ComputeJobHolder job, PlatformMemoryStream stream)
+        {
+            // 1. Unmarshal result.
+            PortableReaderImpl reader = _compute.Marshaller.StartUnmarshal(stream);
+
+            Guid nodeId = reader.ReadGuid().Value;
+            bool cancelled = reader.ReadBoolean();
+
+            try
+            {
+                object err;
+
+                var data = PortableUtils.ReadWrappedInvocationResult(reader, out err);
+
+                // 2. Process the result.
+                return (int) JobResult0(new ComputeJobResultImpl(data, (Exception) err, job.Job, nodeId, cancelled));
+            }
+            catch (Exception e)
+            {
+                Finish(default(TR), e);
+
+                if (!(e is IgniteException))
+                    throw new IgniteException("Failed to process job result: " + e.Message, e);
+
+                throw;
+            }
+        }
+        
+        /** <inheritDoc /> */
+        public void Reduce()
+        {
+            try
+            {
+                TR taskRes = _task.Reduce(_resCache ? _ress : EmptyRes);
+
+                Finish(taskRes, null);
+            }
+            catch (Exception e)
+            {
+                Finish(default(TR), e);
+
+                if (!(e is IgniteException))
+                    throw new IgniteException("Failed to reduce task: " + e.Message, e);
+
+                throw;
+            }
+        }
+
+        /** <inheritDoc /> */
+        public void Complete(long taskHandle)
+        {
+            Clean(taskHandle);
+        }
+
+        /// <summary>
+        /// Complete task with error.
+        /// </summary>
+        /// <param name="taskHandle">Task handle.</param>
+        /// <param name="e">Error.</param>
+        public void CompleteWithError(long taskHandle, Exception e)
+        {
+            Finish(default(TR), e);
+
+            Clean(taskHandle);
+        }
+
+        /** <inheritDoc /> */
+        [SuppressMessage("Microsoft.Design", "CA1031:DoNotCatchGeneralExceptionTypes",
+            Justification = "User object deserialization can throw any exception")]
+        public void CompleteWithError(long taskHandle, PlatformMemoryStream stream)
+        {
+            PortableReaderImpl reader = _compute.Marshaller.StartUnmarshal(stream);
+
+            Exception err;
+
+            try
+            {
+                if (reader.ReadBoolean())
+                {
+                    PortableResultWrapper res = reader.ReadObject<PortableUserObject>()
+                        .Deserialize<PortableResultWrapper>();
+
+                    err = (Exception) res.Result;
+                }
+                else
+                    err = ExceptionUtils.GetException(reader.ReadString(), reader.ReadString());
+            }
+            catch (Exception e)
+            {
+                err = new IgniteException("Task completed with error, but it cannot be unmarshalled: " + e.Message, e);
+            }
+
+            CompleteWithError(taskHandle, err);
+        }
+
+        /// <summary>
+        /// Task completion future.
+        /// </summary>
+        internal IFuture<TR> Future
+        {
+            get { return _fut; }
+        }
+
+        /// <summary>
+        /// Manually set job handles. Used by closures because they have separate flow for map step.
+        /// </summary>
+        /// <param name="jobHandles">Job handles.</param>
+        internal void JobHandles(List<long> jobHandles)
+        {
+            _jobHandles = jobHandles;
+        }
+
+        /// <summary>
+        /// Process job result.
+        /// </summary>
+        /// <param name="res">Result.</param>
+        private ComputeJobResultPolicy JobResult0(IComputeJobResult<object> res)
+        {
+            try
+            {
+                IList<IComputeJobResult<T>> ress0;
+
+                // 1. Prepare old results.
+                if (_resCache)
+                {
+                    if (_resJobs == null)
+                    {
+                        _resJobs = new HashSet<object>();
+
+                        _ress = new List<IComputeJobResult<T>>();
+                    }
+
+                    ress0 = _ress;
+                }
+                else
+                    ress0 = EmptyRes;
+
+                // 2. Invoke user code.
+                var policy = _task.Result(new ComputeJobResultGenericWrapper<T>(res), ress0);
+
+                // 3. Add result to the list only in case of success.
+                if (_resCache)
+                {
+                    var job = res.Job().Unwrap();
+
+                    if (!_resJobs.Add(job))
+                    {
+                        // Duplicate result => find and replace it with the new one.
+                        var oldRes = _ress.Single(item => item.Job() == job);
+
+                        _ress.Remove(oldRes);
+                    }
+
+                    _ress.Add(new ComputeJobResultGenericWrapper<T>(res));
+                }
+
+                return policy;
+            }
+            catch (Exception e)
+            {
+                Finish(default(TR), e);
+
+                if (!(e is IgniteException))
+                    throw new IgniteException("Failed to process job result: " + e.Message, e);
+
+                throw;
+            }
+        }
+
+        /// <summary>
+        /// Finish task.
+        /// </summary>
+        /// <param name="res">Result.</param>
+        /// <param name="err">Error.</param>
+        private void Finish(TR res, Exception err)
+        {
+            _fut.OnDone(res, err);
+        }
+
+        /// <summary>
+        /// Clean-up task resources.
+        /// </summary>
+        /// <param name="taskHandle"></param>
+        private void Clean(long taskHandle)
+        {
+            var handles = _jobHandles;
+
+            var handleRegistry = _compute.Marshaller.Ignite.HandleRegistry;
+
+            if (handles != null)
+                foreach (var handle in handles) 
+                    handleRegistry.Release(handle, true);
+
+            handleRegistry.Release(taskHandle, true);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerBatch.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerBatch.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerBatch.cs
new file mode 100644
index 0000000..cbd26dd
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerBatch.cs
@@ -0,0 +1,269 @@
+/*
+ * 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.Datastream
+{
+    using System;
+    using System.Collections.Concurrent;
+    using System.Collections.Generic;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Threading;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Portable;
+
+    /// <summary>
+    /// Data streamer batch.
+    /// </summary>
+    [SuppressMessage("Microsoft.Design", "CA1001:TypesThatOwnDisposableFieldsShouldBeDisposable")]
+    internal class DataStreamerBatch<TK, TV>
+    {
+        /** Queue. */
+        private readonly ConcurrentQueue<object> _queue = new ConcurrentQueue<object>();
+
+        /** Lock for concurrency. */
+        private readonly ReaderWriterLockSlim _rwLock = new ReaderWriterLockSlim();
+
+        /** Previous batch. */
+        private volatile DataStreamerBatch<TK, TV> _prev;
+
+        /** Current queue size.*/
+        private volatile int _size;
+        
+        /** Send guard. */
+        private bool _sndGuard;
+
+        /** */
+        private readonly Future<object> _fut = new Future<object>();
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        public DataStreamerBatch() : this(null)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="prev">Previous batch.</param>
+        public DataStreamerBatch(DataStreamerBatch<TK, TV> prev)
+        {
+            _prev = prev;
+
+            if (prev != null)
+                Thread.MemoryBarrier(); // Prevent "prev" field escape.
+
+            _fut.Listen(() => ParentsCompleted());
+        }
+
+        /// <summary>
+        /// Gets the future.
+        /// </summary>
+        public IFuture Future
+        {
+            get { return _fut; }
+        }
+
+        /// <summary>
+        /// Add object to the batch.
+        /// </summary>
+        /// <param name="val">Value.</param>
+        /// <param name="cnt">Items count.</param>
+        /// <returns>Positive value in case batch is active, -1 in case no more additions are allowed.</returns>
+        public int Add(object val, int cnt)
+        {
+            // If we cannot enter read-lock immediately, then send is scheduled and batch is definetely blocked.
+            if (!_rwLock.TryEnterReadLock(0))
+                return -1;
+
+            try 
+            {
+                // 1. Ensure additions are possible
+                if (_sndGuard)
+                    return -1;
+
+                // 2. Add data and increase size.
+                _queue.Enqueue(val);
+
+#pragma warning disable 0420
+                int newSize = Interlocked.Add(ref _size, cnt);
+#pragma warning restore 0420
+
+                return newSize;
+            }
+            finally
+            {
+                _rwLock.ExitReadLock();
+            }
+        }
+
+        /// <summary>
+        /// Internal send routine.
+        /// </summary>
+        /// <param name="ldr">streamer.</param>
+        /// <param name="plc">Policy.</param>
+        public void Send(DataStreamerImpl<TK, TV> ldr, int plc)
+        {
+            // 1. Delegate to the previous batch first.
+            DataStreamerBatch<TK, TV> prev0 = _prev;
+
+            if (prev0 != null)
+                prev0.Send(ldr, DataStreamerImpl<TK, TV>.PlcContinue);
+
+            // 2. Set guard.
+            _rwLock.EnterWriteLock();
+
+            try
+            {
+                if (_sndGuard)
+                    return;
+                else
+                    _sndGuard = true;
+            }
+            finally
+            {
+                _rwLock.ExitWriteLock();
+            }
+
+            var handleRegistry = ldr.Marshaller.Ignite.HandleRegistry;
+
+            long futHnd = 0;
+
+            // 3. Actual send.
+            ldr.Update(writer =>
+            {
+                writer.WriteInt(plc);
+
+                if (plc != DataStreamerImpl<TK, TV>.PlcCancelClose)
+                {
+                    futHnd = handleRegistry.Allocate(_fut);
+
+                    try
+                    {
+                        writer.WriteLong(futHnd);
+
+                        WriteTo(writer);
+                    }
+                    catch (Exception)
+                    {
+                        handleRegistry.Release(futHnd);
+
+                        throw;
+                    }
+                }
+            });
+
+            if (plc == DataStreamerImpl<TK, TV>.PlcCancelClose || _size == 0)
+            {
+                _fut.OnNullResult();
+                
+                handleRegistry.Release(futHnd);
+            }
+        }
+
+
+        /// <summary>
+        /// Await completion of current and all previous loads.
+        /// </summary>
+        public void AwaitCompletion()
+        {
+            DataStreamerBatch<TK, TV> curBatch = this;
+
+            while (curBatch != null)
+            {
+                try
+                {
+                    curBatch._fut.Get();
+                }
+                catch (Exception)
+                {
+                    // Ignore.
+                }
+
+                curBatch = curBatch._prev;
+            }
+        }
+
+        /// <summary>
+        /// Write batch content.
+        /// </summary>
+        /// <param name="writer">Portable writer.</param>
+        private void WriteTo(PortableWriterImpl writer)
+        {
+            writer.WriteInt(_size);
+
+            object val;
+
+            while (_queue.TryDequeue(out val))
+            {
+                // 1. Is it a collection?
+                ICollection<KeyValuePair<TK, TV>> entries = val as ICollection<KeyValuePair<TK, TV>>;
+
+                if (entries != null)
+                {
+                    foreach (KeyValuePair<TK, TV> item in entries)
+                    {
+                        writer.Write(item.Key);
+                        writer.Write(item.Value);
+                    }
+
+                    continue;
+                }
+
+                // 2. Is it a single entry?
+                DataStreamerEntry<TK, TV> entry = val as DataStreamerEntry<TK, TV>;
+
+                if (entry != null) {
+                    writer.Write(entry.Key);
+                    writer.Write(entry.Value);
+
+                    continue;
+                }
+
+                // 3. Is it remove merker?
+                DataStreamerRemoveEntry<TK> rmvEntry = val as DataStreamerRemoveEntry<TK>;
+
+                if (rmvEntry != null)
+                {
+                    writer.Write(rmvEntry.Key);
+                    writer.Write<object>(null);
+                }
+            }
+        }
+
+        /// <summary>
+        /// Checck whether all previous batches are completed.
+        /// </summary>
+        /// <returns></returns>
+        private bool ParentsCompleted()
+        {
+            DataStreamerBatch<TK, TV> prev0 = _prev;
+
+            if (prev0 != null)
+            {
+                if (prev0.ParentsCompleted())
+                    _prev = null;
+                else
+                    return false;
+            }
+
+            return _fut.IsDone;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerEntry.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerEntry.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerEntry.cs
new file mode 100644
index 0000000..41ee176
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerEntry.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.Impl.Datastream
+{
+    /// <summary>
+    /// Data streamer entry.
+    /// </summary>
+    internal class DataStreamerEntry<TK, TV>
+    {
+        /** Key. */
+        private readonly TK _key;
+
+        /** Value. */
+        private readonly TV _val;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="key">Key.</param>
+        /// <param name="val">Value.</param>
+        public DataStreamerEntry(TK key, TV val)
+        {
+            _key = key;
+            _val = val;
+        }
+
+        /// <summary>
+        /// Key.
+        /// </summary>
+        public TK Key
+        {
+            get
+            {
+                return _key;
+            }
+        }
+
+        /// <summary>
+        /// Value.
+        /// </summary>
+        public TV Value
+        {
+            get
+            {
+                return _val;
+            }
+        }
+    }
+}


[20/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableWriterImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableWriterImpl.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableWriterImpl.cs
new file mode 100644
index 0000000..c44a0a4
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableWriterImpl.cs
@@ -0,0 +1,1305 @@
+/*
+ * 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.Portable
+{
+    using System;
+    using System.Collections;
+    using System.Collections.Generic;
+    using System.Diagnostics.CodeAnalysis;
+    using System.IO;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Impl.Portable.Metadata;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Portable writer implementation.
+    /// </summary>
+    internal class PortableWriterImpl : IPortableWriter, IPortableRawWriter
+    {
+        /** Marshaller. */
+        private readonly PortableMarshaller _marsh;
+
+        /** Stream. */
+        private readonly IPortableStream _stream;
+
+        /** Builder (used only during build). */
+        private PortableBuilderImpl _builder;
+
+        /** Handles. */
+        private PortableHandleDictionary<object, long> _hnds;
+
+        /** Metadatas collected during this write session. */
+        private IDictionary<int, IPortableMetadata> _metas;
+
+        /** Current type ID. */
+        private int _curTypeId;
+
+        /** Current name converter */
+        private IPortableNameMapper _curConverter;
+
+        /** Current mapper. */
+        private IPortableIdMapper _curMapper;
+
+        /** Current metadata handler. */
+        private IPortableMetadataHandler _curMetaHnd;
+
+        /** Current raw flag. */
+        private bool _curRaw;
+
+        /** Current raw position. */
+        private long _curRawPos;
+
+        /** Ignore handles flag. */
+        private bool _detach;
+
+        /** Object started ignore mode. */
+        private bool _detachMode;
+
+        /// <summary>
+        /// Gets the marshaller.
+        /// </summary>
+        internal PortableMarshaller Marshaller
+        {
+            get { return _marsh; }
+        }
+
+        /// <summary>
+        /// Write named boolean value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Boolean value.</param>
+        public void WriteBoolean(string fieldName, bool val)
+        {
+            WriteSimpleField(fieldName, PortableUtils.TypeBool, val, PortableSystemHandlers.WriteHndBoolTyped, 1);
+        }
+        
+        /// <summary>
+        /// Write boolean value.
+        /// </summary>
+        /// <param name="val">Boolean value.</param>
+        public void WriteBoolean(bool val)
+        {
+            _stream.WriteBool(val);
+        }
+
+        /// <summary>
+        /// Write named boolean array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Boolean array.</param>
+        public void WriteBooleanArray(string fieldName, bool[] val)
+        {
+            WriteSimpleNullableField(fieldName, PortableUtils.TypeArrayBool, val,
+                PortableSystemHandlers.WriteHndBoolArrayTyped, val != null ? val.Length + 4 : 0);
+        }
+
+        /// <summary>
+        /// Write boolean array.
+        /// </summary>
+        /// <param name="val">Boolean array.</param>
+        public void WriteBooleanArray(bool[] val)
+        {
+            WriteSimpleNullableRawField(val, PortableSystemHandlers.WriteHndBoolArrayTyped);
+        }
+
+        /// <summary>
+        /// Write named byte value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Byte value.</param>
+        public void WriteByte(string fieldName, byte val)
+        {
+            WriteSimpleField(fieldName, PortableUtils.TypeByte, val, PortableSystemHandlers.WriteHndByteTyped, 1);
+        }
+
+        /// <summary>
+        /// Write byte value.
+        /// </summary>
+        /// <param name="val">Byte value.</param>
+        public void WriteByte(byte val)
+        {
+            _stream.WriteByte(val);
+        }
+
+        /// <summary>
+        /// Write named byte array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Byte array.</param>
+        public void WriteByteArray(string fieldName, byte[] val)
+        {
+            WriteSimpleNullableField(fieldName, PortableUtils.TypeArrayByte, val,
+                PortableSystemHandlers.WriteHndByteArrayTyped, val != null ? val.Length + 4 : 0);
+        }
+
+        /// <summary>
+        /// Write byte array.
+        /// </summary>
+        /// <param name="val">Byte array.</param>
+        public void WriteByteArray(byte[] val)
+        {
+            WriteSimpleNullableRawField(val, PortableSystemHandlers.WriteHndByteArrayTyped);
+        }
+
+        /// <summary>
+        /// Write named short value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Short value.</param>
+        public void WriteShort(string fieldName, short val)
+        {
+            WriteSimpleField(fieldName, PortableUtils.TypeShort, val, PortableSystemHandlers.WriteHndShortTyped, 2);
+        }
+
+        /// <summary>
+        /// Write short value.
+        /// </summary>
+        /// <param name="val">Short value.</param>
+        public void WriteShort(short val)
+        {
+            _stream.WriteShort(val);
+        }
+
+        /// <summary>
+        /// Write named short array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Short array.</param>
+        public void WriteShortArray(string fieldName, short[] val)
+        {
+            WriteSimpleNullableField(fieldName, PortableUtils.TypeArrayShort, val,
+                PortableSystemHandlers.WriteHndShortArrayTyped, val != null ? 2 * val.Length + 4 : 0);
+        }
+
+        /// <summary>
+        /// Write short array.
+        /// </summary>
+        /// <param name="val">Short array.</param>
+        public void WriteShortArray(short[] val)
+        {
+            WriteSimpleNullableRawField(val, PortableSystemHandlers.WriteHndShortArrayTyped);
+        }
+
+        /// <summary>
+        /// Write named char value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Char value.</param>
+        public void WriteChar(string fieldName, char val)
+        {
+            WriteSimpleField(fieldName, PortableUtils.TypeChar, val, PortableSystemHandlers.WriteHndCharTyped, 2);
+        }
+
+        /// <summary>
+        /// Write char value.
+        /// </summary>
+        /// <param name="val">Char value.</param>
+        public void WriteChar(char val)
+        {
+            _stream.WriteChar(val);
+        }
+
+        /// <summary>
+        /// Write named char array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Char array.</param>
+        public void WriteCharArray(string fieldName, char[] val)
+        {
+            WriteSimpleNullableField(fieldName, PortableUtils.TypeArrayChar, val,
+                PortableSystemHandlers.WriteHndCharArrayTyped, val != null ? 2 * val.Length + 4 : 0);
+        }
+
+        /// <summary>
+        /// Write char array.
+        /// </summary>
+        /// <param name="val">Char array.</param>
+        public void WriteCharArray(char[] val)
+        {
+            WriteSimpleNullableRawField(val, PortableSystemHandlers.WriteHndCharArrayTyped);
+        }
+
+        /// <summary>
+        /// Write named int value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Int value.</param>
+        public void WriteInt(string fieldName, int val)
+        {
+            WriteSimpleField(fieldName, PortableUtils.TypeInt, val, PortableSystemHandlers.WriteHndIntTyped, 4);
+        }
+
+        /// <summary>
+        /// Write int value.
+        /// </summary>
+        /// <param name="val">Int value.</param>
+        public void WriteInt(int val)
+        {
+            _stream.WriteInt(val);
+        }
+
+        /// <summary>
+        /// Write named int array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Int array.</param>
+        public void WriteIntArray(string fieldName, int[] val)
+        {
+            WriteSimpleNullableField(fieldName, PortableUtils.TypeArrayInt, val,
+                PortableSystemHandlers.WriteHndIntArrayTyped, val != null ? 4 * val.Length + 4 : 0);
+        }
+
+        /// <summary>
+        /// Write int array.
+        /// </summary>
+        /// <param name="val">Int array.</param>
+        public void WriteIntArray(int[] val)
+        {
+            WriteSimpleNullableRawField(val, PortableSystemHandlers.WriteHndIntArrayTyped);
+        }
+
+        /// <summary>
+        /// Write named long value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Long value.</param>
+        public void WriteLong(string fieldName, long val)
+        {
+            WriteSimpleField(fieldName, PortableUtils.TypeLong, val, PortableSystemHandlers.WriteHndLongTyped, 8);
+        }
+
+        /// <summary>
+        /// Write long value.
+        /// </summary>
+        /// <param name="val">Long value.</param>
+        public void WriteLong(long val)
+        {
+            _stream.WriteLong(val);
+        }
+
+        /// <summary>
+        /// Write named long array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Long array.</param>
+        public void WriteLongArray(string fieldName, long[] val)
+        {
+            WriteSimpleNullableField(fieldName, PortableUtils.TypeArrayLong, val,
+                PortableSystemHandlers.WriteHndLongArrayTyped, val != null ? 8 * val.Length + 4 : 0);
+        }
+
+        /// <summary>
+        /// Write long array.
+        /// </summary>
+        /// <param name="val">Long array.</param>
+        public void WriteLongArray(long[] val)
+        {
+            WriteSimpleNullableRawField(val, PortableSystemHandlers.WriteHndLongArrayTyped);
+        }
+
+        /// <summary>
+        /// Write named float value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Float value.</param>
+        public void WriteFloat(string fieldName, float val)
+        {
+            WriteSimpleField(fieldName, PortableUtils.TypeFloat, val, PortableSystemHandlers.WriteHndFloatTyped, 4);
+        }
+
+        /// <summary>
+        /// Write float value.
+        /// </summary>
+        /// <param name="val">Float value.</param>
+        public void WriteFloat(float val)
+        {
+            _stream.WriteFloat(val);
+        }
+
+        /// <summary>
+        /// Write named float array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Float array.</param>
+        public void WriteFloatArray(string fieldName, float[] val)
+        {
+            WriteSimpleNullableField(fieldName, PortableUtils.TypeArrayFloat, val,
+                PortableSystemHandlers.WriteHndFloatArrayTyped, val != null ? 4 * val.Length + 4 : 0);
+        }
+
+        /// <summary>
+        /// Write float array.
+        /// </summary>
+        /// <param name="val">Float array.</param>
+        public void WriteFloatArray(float[] val)
+        {
+            WriteSimpleNullableRawField(val, PortableSystemHandlers.WriteHndFloatArrayTyped);
+        }
+
+        /// <summary>
+        /// Write named double value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Double value.</param>
+        public void WriteDouble(string fieldName, double val)
+        {
+            WriteSimpleField(fieldName, PortableUtils.TypeDouble, val, PortableSystemHandlers.WriteHndDoubleTyped, 8);
+        }
+
+        /// <summary>
+        /// Write double value.
+        /// </summary>
+        /// <param name="val">Double value.</param>
+        public void WriteDouble(double val)
+        {
+            _stream.WriteDouble(val);
+        }
+
+        /// <summary>
+        /// Write named double array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Double array.</param>
+        public void WriteDoubleArray(string fieldName, double[] val)
+        {
+            WriteSimpleNullableField(fieldName, PortableUtils.TypeArrayDouble, val,
+                PortableSystemHandlers.WriteHndDoubleArrayTyped, val != null ? 8 * val.Length + 4 : 0);
+        }
+
+        /// <summary>
+        /// Write double array.
+        /// </summary>
+        /// <param name="val">Double array.</param>
+        public void WriteDoubleArray(double[] val)
+        {
+            WriteSimpleNullableRawField(val, PortableSystemHandlers.WriteHndDoubleArrayTyped);
+        }
+
+        /// <summary>
+        /// Write named decimal value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Decimal value.</param>
+        public void WriteDecimal(string fieldName, decimal val)
+        {
+            WriteSimpleNullableField(fieldName, PortableUtils.TypeDecimal, val, PortableSystemHandlers.WriteHndDecimalTyped);
+        }
+
+        /// <summary>
+        /// Write decimal value.
+        /// </summary>
+        /// <param name="val">Decimal value.</param>
+        public void WriteDecimal(decimal val)
+        {
+            WriteSimpleNullableRawField(val, PortableSystemHandlers.WriteHndDecimalTyped);
+        }
+
+        /// <summary>
+        /// Write named decimal array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Decimal array.</param>
+        public void WriteDecimalArray(string fieldName, decimal[] val)
+        {
+            WriteSimpleNullableField(fieldName, PortableUtils.TypeArrayDecimal, val,
+                PortableSystemHandlers.WriteHndDecimalArrayTyped);
+        }
+        
+        /// <summary>
+        /// Write decimal array.
+        /// </summary>
+        /// <param name="val">Decimal array.</param>
+        public void WriteDecimalArray(decimal[] val)
+        {
+            WriteSimpleNullableRawField(val, PortableSystemHandlers.WriteHndDecimalArrayTyped);
+        }
+
+        /// <summary>
+        /// Write named date value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Date value.</param>
+        public void WriteDate(string fieldName, DateTime? val)
+        {
+            WriteSimpleNullableField(fieldName, PortableUtils.TypeDate, val, PortableSystemHandlers.WriteHndDateTyped,
+                val.HasValue ? 12 : 0);
+        }
+        
+        /// <summary>
+        /// Write date value.
+        /// </summary>
+        /// <param name="val">Date value.</param>
+        public void WriteDate(DateTime? val)
+        {
+            WriteSimpleNullableRawField(val, PortableSystemHandlers.WriteHndDateTyped);
+        }
+
+        /// <summary>
+        /// Write named date array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Date array.</param>
+        public void WriteDateArray(string fieldName, DateTime?[] val)
+        {
+            WriteSimpleNullableField(fieldName, PortableUtils.TypeArrayDate, val,
+                PortableSystemHandlers.WriteHndDateArrayTyped);
+        }
+
+        /// <summary>
+        /// Write date array.
+        /// </summary>
+        /// <param name="val">Date array.</param>
+        public void WriteDateArray(DateTime?[] val)
+        {
+            WriteSimpleNullableRawField(val, PortableSystemHandlers.WriteHndDateArrayTyped);
+        }
+
+        /// <summary>
+        /// Write named string value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">String value.</param>
+        public void WriteString(string fieldName, string val)
+        {
+            WriteSimpleNullableField(fieldName, PortableUtils.TypeString, val, PortableSystemHandlers.WriteHndStringTyped);
+        }
+
+        /// <summary>
+        /// Write string value.
+        /// </summary>
+        /// <param name="val">String value.</param>
+        public void WriteString(string val)
+        {
+            WriteSimpleNullableRawField(val, PortableSystemHandlers.WriteHndStringTyped);
+        }
+
+        /// <summary>
+        /// Write named string array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">String array.</param>
+        public void WriteStringArray(string fieldName, string[] val)
+        {
+            WriteSimpleNullableField(fieldName, PortableUtils.TypeArrayString, val,
+                PortableSystemHandlers.WriteHndStringArrayTyped);
+        }
+
+        /// <summary>
+        /// Write string array.
+        /// </summary>
+        /// <param name="val">String array.</param>
+        public void WriteStringArray(string[] val)
+        {
+            WriteSimpleNullableRawField(val, PortableSystemHandlers.WriteHndStringArrayTyped);
+        }
+
+        /// <summary>
+        /// Write named GUID value.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">GUID value.</param>
+        public void WriteGuid(string fieldName, Guid? val)
+        {
+            WriteSimpleNullableField(fieldName, PortableUtils.TypeGuid, val, PortableSystemHandlers.WriteHndGuidTyped,
+                val.HasValue ? 16 : 0);
+        }
+
+        /// <summary>
+        /// Write GUID value.
+        /// </summary>
+        /// <param name="val">GUID value.</param>
+        public void WriteGuid(Guid? val)
+        {
+            WriteSimpleNullableRawField(val, PortableSystemHandlers.WriteHndGuidTyped);
+        }
+
+        /// <summary>
+        /// Write named GUID array.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">GUID array.</param>
+        public void WriteGuidArray(string fieldName, Guid?[] val)
+        {
+            WriteSimpleNullableField(fieldName, PortableUtils.TypeArrayGuid, val,
+                PortableSystemHandlers.WriteHndGuidArrayTyped);
+        }
+
+        /// <summary>
+        /// Write GUID array.
+        /// </summary>
+        /// <param name="val">GUID array.</param>
+        public void WriteGuidArray(Guid?[] val)
+        {
+            WriteSimpleNullableRawField(val, PortableSystemHandlers.WriteHndGuidArrayTyped);
+        }
+
+        /// <summary>
+        /// Write named enum value.
+        /// </summary>
+        /// <typeparam name="T"></typeparam>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Enum value.</param>
+        public void WriteEnum<T>(string fieldName, T val)
+        {
+            WriteField(fieldName, PortableUtils.TypeEnum, val, PortableSystemHandlers.WriteHndEnum);
+        }
+
+        /// <summary>
+        /// Write enum value.
+        /// </summary>
+        /// <typeparam name="T"></typeparam>
+        /// <param name="val">Enum value.</param>
+        public void WriteEnum<T>(T val)
+        {
+            Write(val, PortableSystemHandlers.WriteHndEnum);
+        }
+
+        /// <summary>
+        /// Write named enum array.
+        /// </summary>
+        /// <typeparam name="T"></typeparam>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Enum array.</param>
+        public void WriteEnumArray<T>(string fieldName, T[] val)
+        {
+            WriteField(fieldName, PortableUtils.TypeArrayEnum, val, PortableSystemHandlers.WriteHndEnumArray);
+        }
+
+        /// <summary>
+        /// Write enum array.
+        /// </summary>
+        /// <typeparam name="T"></typeparam>
+        /// <param name="val">Enum array.</param>
+        public void WriteEnumArray<T>(T[] val)
+        {
+            Write(val, PortableSystemHandlers.WriteHndEnumArray);
+        }
+
+        /// <summary>
+        /// Write named object value.
+        /// </summary>
+        /// <typeparam name="T"></typeparam>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Object value.</param>
+        public void WriteObject<T>(string fieldName, T val)
+        {
+            WriteField(fieldName, PortableUtils.TypeObject, val, null);
+        }
+
+        /// <summary>
+        /// Write object value.
+        /// </summary>
+        /// <typeparam name="T"></typeparam>
+        /// <param name="val">Object value.</param>
+        public void WriteObject<T>(T val)
+        {
+            Write(val);
+        }
+
+        /// <summary>
+        /// Write named object array.
+        /// </summary>
+        /// <typeparam name="T"></typeparam>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Object array.</param>
+        public void WriteObjectArray<T>(string fieldName, T[] val)
+        {
+            WriteField(fieldName, PortableUtils.TypeArray, val, PortableSystemHandlers.WriteHndArray);
+        }
+
+        /// <summary>
+        /// Write object array.
+        /// </summary>
+        /// <typeparam name="T"></typeparam>
+        /// <param name="val">Object array.</param>
+        public void WriteObjectArray<T>(T[] val)
+        {
+            Write(val, PortableSystemHandlers.WriteHndArray);
+        }
+
+        /// <summary>
+        /// Write named collection.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Collection.</param>
+        public void WriteCollection(string fieldName, ICollection val)
+        {
+            WriteField(fieldName, PortableUtils.TypeCollection, val, null);
+        }
+
+        /// <summary>
+        /// Write collection.
+        /// </summary>
+        /// <param name="val">Collection.</param>
+        public void WriteCollection(ICollection val)
+        {
+            Write(val);
+        }
+
+        /// <summary>
+        /// Write named generic collection.
+        /// </summary>
+        /// <typeparam name="T"></typeparam>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Collection.</param>
+        public void WriteGenericCollection<T>(string fieldName, ICollection<T> val)
+        {
+            WriteField(fieldName, PortableUtils.TypeCollection, val, null);
+        }
+
+        /// <summary>
+        /// Write generic collection.
+        /// </summary>
+        /// <typeparam name="T"></typeparam>
+        /// <param name="val">Collection.</param>
+        public void WriteGenericCollection<T>(ICollection<T> val)
+        {
+            Write(val);
+        }
+
+        /// <summary>
+        /// Write named dictionary.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Dictionary.</param>
+        public void WriteDictionary(string fieldName, IDictionary val)
+        {
+            WriteField(fieldName, PortableUtils.TypeDictionary, val, null);
+        }
+
+        /// <summary>
+        /// Write dictionary.
+        /// </summary>
+        /// <param name="val">Dictionary.</param>
+        public void WriteDictionary(IDictionary val)
+        {
+            Write(val);
+        }
+
+        /// <summary>
+        /// Write named generic dictionary.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Dictionary.</param>
+        public void WriteGenericDictionary<TK, TV>(string fieldName, IDictionary<TK, TV> val)
+        {
+            WriteField(fieldName, PortableUtils.TypeDictionary, val, null);
+        }
+
+        /// <summary>
+        /// Write generic dictionary.
+        /// </summary>
+        /// <param name="val">Dictionary.</param>
+        public void WriteGenericDictionary<TK, TV>(IDictionary<TK, TV> val)
+        {
+            Write(val);
+        }
+
+        /// <summary>
+        /// Get raw writer.
+        /// </summary>
+        /// <returns>
+        /// Raw writer.
+        /// </returns>
+        public IPortableRawWriter RawWriter()
+        {
+            if (!_curRaw)
+            {
+                _curRaw = true;
+                _curRawPos = _stream.Position;
+            }
+
+            return this;
+        }
+
+        /// <summary>
+        /// Set new builder.
+        /// </summary>
+        /// <param name="builder">Builder.</param>
+        /// <returns>Previous builder.</returns>
+        internal PortableBuilderImpl Builder(PortableBuilderImpl builder)
+        {
+            PortableBuilderImpl ret = _builder;
+
+            _builder = builder;
+
+            return ret;
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="marsh">Marshaller.</param>
+        /// <param name="stream">Stream.</param>
+        internal PortableWriterImpl(PortableMarshaller marsh, IPortableStream stream)
+        {
+            _marsh = marsh;
+            _stream = stream;
+        }
+
+        /// <summary>
+        /// Write object.
+        /// </summary>
+        /// <param name="obj">Object.</param>
+        internal void Write<T>(T obj)
+        {
+            Write(obj, null);
+        }
+
+        /// <summary>
+        /// Write object.
+        /// </summary>
+        /// <param name="obj">Object.</param>
+        /// <param name="handler">Optional write handler.</param>
+        [SuppressMessage("ReSharper", "FunctionComplexityOverflow")]
+        internal void Write<T>(T obj, object handler)
+        {
+            // Apply detach mode if needed.
+            PortableHandleDictionary<object, long> oldHnds = null;
+
+            bool resetDetach = false;
+
+            if (_detach)
+            {
+                _detach = false;
+                _detachMode = true;
+                resetDetach = true;
+
+                oldHnds = _hnds;
+
+                _hnds = null;
+            }
+
+            try
+            {
+                // Write null.
+                if (obj == null)
+                {
+                    _stream.WriteByte(PortableUtils.HdrNull);
+
+                    return;
+                }
+
+                if (_builder != null)
+                {
+                    // Special case for portable object during build.
+                    PortableUserObject portObj = obj as PortableUserObject;
+
+                    if (portObj != null)
+                    {
+                        if (!WriteHandle(_stream.Position, portObj))
+                            _builder.ProcessPortable(_stream, portObj);
+
+                        return;
+                    }
+
+                    // Special case for builder during build.
+                    PortableBuilderImpl portBuilder = obj as PortableBuilderImpl;
+
+                    if (portBuilder != null)
+                    {
+                        if (!WriteHandle(_stream.Position, portBuilder))
+                            _builder.ProcessBuilder(_stream, portBuilder);
+
+                        return;
+                    }
+                }                
+
+                // Try writting as well-known type.
+                if (InvokeHandler(handler, handler as PortableSystemWriteDelegate, obj))
+                    return;
+
+                Type type = obj.GetType();
+
+                IPortableTypeDescriptor desc = _marsh.Descriptor(type);
+
+                object typedHandler;
+                PortableSystemWriteDelegate untypedHandler;
+
+                if (desc == null)
+                {
+                    typedHandler = null;
+                    untypedHandler = PortableSystemHandlers.WriteHandler(type);
+                }
+                else
+                {
+                    typedHandler = desc.TypedHandler;
+                    untypedHandler = desc.UntypedHandler;
+                }
+
+                if (InvokeHandler(typedHandler, untypedHandler, obj))
+                    return;
+
+                if (desc == null)
+                {
+                    if (!type.IsSerializable)
+                        // If neither handler, nor descriptor exist, and not serializable, this is an exception.
+                        throw new PortableException("Unsupported object type [type=" + type +
+                            ", object=" + obj + ']');
+
+                    Write(new SerializableObjectHolder(obj));
+
+                    return;
+                }
+
+                int pos = _stream.Position;
+
+                // Dealing with handles.
+                if (!(desc.Serializer is IPortableSystemTypeSerializer) && WriteHandle(pos, obj))
+                    return;
+
+                // Write header.
+                _stream.WriteByte(PortableUtils.HdrFull);
+
+                _stream.WriteBool(desc.UserType);
+                _stream.WriteInt(desc.TypeId);
+                _stream.WriteInt(obj.GetHashCode());
+
+                // Skip length as it is not known in the first place.
+                _stream.Seek(8, SeekOrigin.Current);
+
+                // Preserve old frame.
+                int oldTypeId = _curTypeId;
+                IPortableNameMapper oldConverter = _curConverter;
+                IPortableIdMapper oldMapper = _curMapper;
+                IPortableMetadataHandler oldMetaHnd = _curMetaHnd;
+                bool oldRaw = _curRaw;
+                long oldRawPos = _curRawPos;
+
+                // Push new frame.
+                _curTypeId = desc.TypeId;
+                _curConverter = desc.NameConverter;
+                _curMapper = desc.Mapper;
+                _curMetaHnd = desc.MetadataEnabled ? _marsh.MetadataHandler(desc) : null;
+                _curRaw = false;
+                _curRawPos = 0;
+
+                // Write object fields.
+                desc.Serializer.WritePortable(obj, this);
+
+                // Calculate and write length.
+                int retPos = _stream.Position;
+
+                _stream.Seek(pos + 10, SeekOrigin.Begin);
+
+                int len = retPos - pos;
+
+                _stream.WriteInt(len);
+
+                if (_curRawPos != 0)
+                    // When set, it is difference between object head and raw position.
+                    _stream.WriteInt((int)(_curRawPos - pos));
+                else
+                    // When no set, it is equal to object length.
+                    _stream.WriteInt(len);
+
+                _stream.Seek(retPos, SeekOrigin.Begin);
+
+                // 13. Collect metadata.
+                if (_curMetaHnd != null)
+                {
+                    IDictionary<string, int> meta = _curMetaHnd.OnObjectWriteFinished();
+
+                    if (meta != null)
+                        SaveMetadata(_curTypeId, desc.TypeName, desc.AffinityKeyFieldName, meta);
+                }
+
+                // Restore old frame.
+                _curTypeId = oldTypeId;
+                _curConverter = oldConverter;
+                _curMapper = oldMapper;
+                _curMetaHnd = oldMetaHnd;
+                _curRaw = oldRaw;
+                _curRawPos = oldRawPos;
+            }
+            finally
+            {
+                // Restore handles if needed.
+                if (resetDetach)
+                {
+                    // Add newly recorded handles without overriding already existing ones.
+                    if (_hnds != null)
+                    {
+                        if (oldHnds == null)
+                            oldHnds = _hnds;
+                        else
+                            oldHnds.Merge(_hnds);
+                    }
+
+                    _hnds = oldHnds;
+
+                    _detachMode = false;
+                }
+            }
+        }
+
+        /// <summary>
+        /// Add handle to handles map.
+        /// </summary>
+        /// <param name="pos">Position in stream.</param>
+        /// <param name="obj">Object.</param>
+        /// <returns><c>true</c> if object was written as handle.</returns>
+        private bool WriteHandle(long pos, object obj)
+        {
+            if (_hnds == null)
+            {
+                // Cache absolute handle position.
+                _hnds = new PortableHandleDictionary<object, long>(obj, pos);
+
+                return false;
+            }
+
+            long hndPos;
+
+            if (!_hnds.TryGetValue(obj, out hndPos))
+            {
+                // Cache absolute handle position.
+                _hnds.Add(obj, pos);
+
+                return false;
+            }
+
+            _stream.WriteByte(PortableUtils.HdrHnd);
+
+            // Handle is written as difference between position before header and handle position.
+            _stream.WriteInt((int)(pos - hndPos));
+
+            return true;
+        }
+
+        /// <summary>
+        /// Try invoking predefined handler on object.
+        /// </summary>
+        /// <param name="typedHandler">Handler</param>
+        /// <param name="untypedHandler">Not typed handler.</param>
+        /// <param name="obj">Object.</param>
+        /// <returns>True if handler was called.</returns>
+        private bool InvokeHandler<T>(object typedHandler, PortableSystemWriteDelegate untypedHandler, T obj)
+        {
+            var typedHandler0 = typedHandler as PortableSystemTypedWriteDelegate<T>;
+
+            if (typedHandler0 != null)
+            {
+                typedHandler0.Invoke(_stream, obj);
+
+                return true;
+            }
+
+            if (untypedHandler != null)
+            {
+                untypedHandler.Invoke(this, obj);
+
+                return true;
+            }
+
+            return false;
+        }
+
+        /// <summary>
+        /// Write simple field with known length.
+        /// </summary>
+        /// <param name="fieldId">Field ID.</param>
+        /// <param name="val">Value.</param>
+        /// <param name="handler">Handler.</param>
+        /// <param name="len">Length.</param>
+        private void WriteSimpleField<T>(int fieldId, T val, PortableSystemTypedWriteDelegate<T> handler, int len)
+        {
+            CheckNotRaw();
+
+            _stream.WriteInt(fieldId);
+            _stream.WriteInt(1 + len); // Additional byte for field type.
+
+            handler(_stream, val);
+        }
+
+        /// <summary>
+        /// Write simple nullable field with unknown length.
+        /// </summary>
+        /// <param name="fieldId">Field ID.</param>
+        /// <param name="val">Value.</param>
+        /// <param name="handler">Handler.</param>
+        private void WriteSimpleNullableField<T>(int fieldId, T val, PortableSystemTypedWriteDelegate<T> handler)
+        {
+            CheckNotRaw();
+
+            _stream.WriteInt(fieldId);
+
+            if (val == null)
+            {
+                _stream.WriteInt(1);
+
+                _stream.WriteByte(PortableUtils.HdrNull);
+            }
+            else
+            {
+                int pos = _stream.Position;
+
+                _stream.Seek(4, SeekOrigin.Current);
+
+                handler(_stream, val);
+
+                WriteFieldLength(_stream, pos);
+            }
+        }
+
+        /// <summary>
+        /// Write simple nullable field with known length.
+        /// </summary>
+        /// <param name="fieldId">Field ID.</param>
+        /// <param name="val">Value.</param>
+        /// <param name="handler">Handler.</param>
+        /// <param name="len">Length.</param>
+        private void WriteSimpleNullableField<T>(int fieldId, T val, PortableSystemTypedWriteDelegate<T> handler, int len)
+        {
+            CheckNotRaw();
+
+            _stream.WriteInt(fieldId);
+
+            if (val == null)
+            {
+                _stream.WriteInt(1);
+
+                _stream.WriteByte(PortableUtils.HdrNull);
+            }
+            else
+            {
+                _stream.WriteInt(1 + len);
+
+                handler(_stream, val);
+            }
+        }
+
+        /// <summary>
+        /// Write field.
+        /// </summary>
+        /// <param name="fieldId">Field ID.</param>
+        /// <param name="val">Value.</param>
+        /// <param name="handler">Handler.</param>
+        private void WriteField(int fieldId, object val, PortableSystemWriteDelegate handler)
+        {
+            CheckNotRaw();
+
+            _stream.WriteInt(fieldId);
+
+            int pos = _stream.Position;
+
+            _stream.Seek(4, SeekOrigin.Current);
+
+            Write(val, handler);
+
+            WriteFieldLength(_stream, pos);
+        }
+
+        /// <summary>
+        /// Enable detach mode for the next object.
+        /// </summary>
+        internal void DetachNext()
+        {
+            if (!_detachMode)
+                _detach = true;
+        }
+
+        /// <summary>
+        /// Stream.
+        /// </summary>
+        internal IPortableStream Stream
+        {
+            get { return _stream; }
+        }
+
+        /// <summary>
+        /// Gets collected metadatas.
+        /// </summary>
+        /// <returns>Collected metadatas (if any).</returns>
+        internal IDictionary<int, IPortableMetadata> Metadata()
+        {
+            return _metas;
+        }
+
+        /// <summary>
+        /// Check whether the given object is portable, i.e. it can be 
+        /// serialized with portable marshaller.
+        /// </summary>
+        /// <param name="obj">Object.</param>
+        /// <returns>True if portable.</returns>
+        internal bool IsPortable(object obj)
+        {
+            if (obj != null)
+            {
+                Type type = obj.GetType();
+
+                // We assume object as portable only in case it has descriptor.
+                // Collections, Enums and non-primitive arrays do not have descriptors
+                // and this is fine here because we cannot know whether their members
+                // are portable.
+                return _marsh.Descriptor(type) != null;
+            }
+
+            return true;
+        }
+
+        /// <summary>
+        /// Write simple field with known length.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="typeId">Type ID.</param>
+        /// <param name="val">Value.</param>
+        /// <param name="handler">Handler.</param>
+        /// <param name="len">Length.</param>
+        private void WriteSimpleField<T>(string fieldName, byte typeId, T val,
+            PortableSystemTypedWriteDelegate<T> handler, int len)
+        {
+            int fieldId = PortableUtils.FieldId(_curTypeId, fieldName, _curConverter, _curMapper);
+
+            WriteSimpleField(fieldId, val, handler, len);
+
+            if (_curMetaHnd != null)
+                _curMetaHnd.OnFieldWrite(fieldId, fieldName, typeId);
+        }
+
+        /// <summary>
+        /// Write simple nullable field with unknown length.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="typeId">Type ID.</param>
+        /// <param name="val">Value.</param>
+        /// <param name="handler">Handler.</param>
+        private void WriteSimpleNullableField<T>(string fieldName, byte typeId, T val,
+            PortableSystemTypedWriteDelegate<T> handler)
+        {
+            int fieldId = PortableUtils.FieldId(_curTypeId, fieldName, _curConverter, _curMapper);
+
+            WriteSimpleNullableField(fieldId, val, handler);
+
+            if (_curMetaHnd != null)
+                _curMetaHnd.OnFieldWrite(fieldId, fieldName, typeId);
+        }
+
+        /// <summary>
+        /// Write simple nullable field with known length.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="typeId">Type ID.</param>
+        /// <param name="val">Value.</param>
+        /// <param name="handler">Handler.</param>
+        /// <param name="len">Length.</param>
+        private void WriteSimpleNullableField<T>(string fieldName, byte typeId, T val,
+            PortableSystemTypedWriteDelegate<T> handler, int len)
+        {
+            int fieldId = PortableUtils.FieldId(_curTypeId, fieldName, _curConverter, _curMapper);
+
+            WriteSimpleNullableField(fieldId, val, handler, len);
+
+            if (_curMetaHnd != null)
+                _curMetaHnd.OnFieldWrite(fieldId, fieldName, typeId);
+        }
+
+        /// <summary>
+        /// Write nullable raw field.
+        /// </summary>
+        /// <param name="val">Value.</param>
+        /// <param name="handler">Handler.</param>
+        private void WriteSimpleNullableRawField<T>(T val, PortableSystemTypedWriteDelegate<T> handler)
+        {
+            if (val == null)
+                _stream.WriteByte(PortableUtils.HdrNull);
+            else
+                handler(_stream, val);
+        }
+
+        /// <summary>
+        /// Write field.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="typeId">Type ID.</param>
+        /// <param name="val">Value.</param>
+        /// <param name="handler">Handler.</param>
+        private void WriteField(string fieldName, byte typeId, object val,
+            PortableSystemWriteDelegate handler)
+        {
+            int fieldId = PortableUtils.FieldId(_curTypeId, fieldName, _curConverter, _curMapper);
+
+            WriteField(fieldId, val, handler);
+
+            if (_curMetaHnd != null)
+                _curMetaHnd.OnFieldWrite(fieldId, fieldName, typeId);
+        }
+
+        /// <summary>
+        /// Write field length.
+        /// </summary>
+        /// <param name="stream">Stream.</param>
+        /// <param name="pos">Position where length should reside</param>
+        private static void WriteFieldLength(IPortableStream stream, int pos)
+        {
+            int retPos = stream.Position;
+
+            stream.Seek(pos, SeekOrigin.Begin);
+
+            stream.WriteInt(retPos - pos - 4);
+
+            stream.Seek(retPos, SeekOrigin.Begin);
+        }
+
+        /// <summary>
+        /// Ensure that we are not in raw mode.
+        /// </summary>
+        private void CheckNotRaw()
+        {
+            if (_curRaw)
+                throw new PortableException("Cannot write named fields after raw data is written.");
+        }
+
+        /// <summary>
+        /// Saves metadata for this session.
+        /// </summary>
+        /// <param name="typeId">Type ID.</param>
+        /// <param name="typeName">Type name.</param>
+        /// <param name="affKeyFieldName">Affinity key field name.</param>
+        /// <param name="fields">Fields metadata.</param>
+        internal void SaveMetadata(int typeId, string typeName, string affKeyFieldName, IDictionary<string, int> fields)
+        {
+            if (_metas == null)
+            {
+                PortableMetadataImpl meta =
+                    new PortableMetadataImpl(typeId, typeName, fields, affKeyFieldName);
+
+                _metas = new Dictionary<int, IPortableMetadata>(1);
+
+                _metas[typeId] = meta;
+            }
+            else
+            {
+                IPortableMetadata meta;
+
+                if (_metas.TryGetValue(typeId, out meta))
+                {
+                    IDictionary<string, int> existingFields = ((PortableMetadataImpl)meta).FieldsMap();
+
+                    foreach (KeyValuePair<string, int> field in fields)
+                    {
+                        if (!existingFields.ContainsKey(field.Key))
+                            existingFields[field.Key] = field.Value;
+                    }
+                }
+                else
+                    _metas[typeId] = new PortableMetadataImpl(typeId, typeName, fields, affKeyFieldName);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortablesImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortablesImpl.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortablesImpl.cs
new file mode 100644
index 0000000..066f46b
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortablesImpl.cs
@@ -0,0 +1,205 @@
+/*
+ * 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.Portable
+{
+    using System;
+    using System.Collections.Generic;
+    using System.IO;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Portables implementation.
+    /// </summary>
+    internal class PortablesImpl : IPortables
+    {
+        /** Owning grid. */
+        private readonly PortableMarshaller _marsh;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="marsh">Marshaller.</param>
+        internal PortablesImpl(PortableMarshaller marsh)
+        {
+            _marsh = marsh;
+        }
+
+        /** <inheritDoc /> */
+        public T ToPortable<T>(object obj)
+        {
+            if (obj is IPortableObject)
+                return (T)obj;
+
+            IPortableStream stream = new PortableHeapStream(1024);
+
+            // Serialize.
+            PortableWriterImpl writer = _marsh.StartMarshal(stream);
+
+            try
+            {
+                writer.Write(obj);
+            }
+            finally
+            {
+                // Save metadata.
+                _marsh.FinishMarshal(writer);
+            }
+
+            // Deserialize.
+            stream.Seek(0, SeekOrigin.Begin);
+
+            return _marsh.Unmarshal<T>(stream, PortableMode.ForcePortable);
+        }
+
+        /** <inheritDoc /> */
+        public IPortableBuilder Builder(Type type)
+        {
+            IgniteArgumentCheck.NotNull(type, "type");
+
+            IPortableTypeDescriptor desc = _marsh.Descriptor(type);
+
+            if (desc == null)
+                throw new IgniteException("Type is not portable (add it to PortableConfiguration): " + 
+                    type.FullName);
+
+            return Builder0(null, PortableFromDescriptor(desc), desc);
+        }
+
+        /** <inheritDoc /> */
+        public IPortableBuilder Builder(string typeName)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(typeName, "typeName");
+
+            IPortableTypeDescriptor desc = _marsh.Descriptor(typeName);
+            
+            return Builder0(null, PortableFromDescriptor(desc), desc);
+        }
+
+        /** <inheritDoc /> */
+        public IPortableBuilder Builder(IPortableObject obj)
+        {
+            IgniteArgumentCheck.NotNull(obj, "obj");
+
+            PortableUserObject obj0 = obj as PortableUserObject;
+
+            if (obj0 == null)
+                throw new ArgumentException("Unsupported object type: " + obj.GetType());
+
+            IPortableTypeDescriptor desc = _marsh.Descriptor(true, obj0.TypeId());
+            
+            return Builder0(null, obj0, desc);
+        }
+
+        /** <inheritDoc /> */
+        public int GetTypeId(string typeName)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(typeName, "typeName");
+
+            return Marshaller.Descriptor(typeName).TypeId;
+        }
+
+        /** <inheritDoc /> */
+        public ICollection<IPortableMetadata> GetMetadata()
+        {
+            return Marshaller.Ignite.ClusterGroup.Metadata();
+        }
+
+        /** <inheritDoc /> */
+        public IPortableMetadata GetMetadata(int typeId)
+        {
+            return Marshaller.Metadata(typeId);
+        }
+
+        /** <inheritDoc /> */
+        public IPortableMetadata GetMetadata(string typeName)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(typeName, "typeName");
+
+            return GetMetadata(GetTypeId(typeName));
+        }
+
+        /** <inheritDoc /> */
+        public IPortableMetadata GetMetadata(Type type)
+        {
+            IgniteArgumentCheck.NotNull(type, "type");
+
+            var desc = Marshaller.Descriptor(type);
+
+            return desc == null ? null : Marshaller.Metadata(desc.TypeId);
+        }
+
+        /// <summary>
+        /// Create child builder.
+        /// </summary>
+        /// <param name="parent">Parent builder.</param>
+        /// <param name="obj">Portable object.</param>
+        /// <returns></returns>
+        internal PortableBuilderImpl ChildBuilder(PortableBuilderImpl parent, PortableUserObject obj)
+        {
+            IPortableTypeDescriptor desc = _marsh.Descriptor(true, obj.TypeId());
+
+            return Builder0(null, obj, desc);
+        }
+
+        /// <summary>
+        /// Marshaller.
+        /// </summary>
+        internal PortableMarshaller Marshaller
+        {
+            get
+            {
+                return _marsh;
+            }
+        }
+
+        /// <summary>
+        /// Create empty portable object from descriptor.
+        /// </summary>
+        /// <param name="desc">Descriptor.</param>
+        /// <returns>Empty portable object.</returns>
+        private PortableUserObject PortableFromDescriptor(IPortableTypeDescriptor desc)
+        {
+            PortableHeapStream stream = new PortableHeapStream(18);
+
+            stream.WriteByte(PortableUtils.HdrFull);
+            stream.WriteBool(true);
+            stream.WriteInt(desc.TypeId);
+            stream.WriteInt(0); // Hash.
+            stream.WriteInt(PortableUtils.FullHdrLen); // Length.
+            stream.WriteInt(PortableUtils.FullHdrLen); // Raw data offset.
+
+            return new PortableUserObject(_marsh, stream.InternalArray, 0, desc.TypeId, 0);
+        }
+
+        /// <summary>
+        /// Internal builder creation routine.
+        /// </summary>
+        /// <param name="parent">Parent builder.</param>
+        /// <param name="obj">Portable object.</param>
+        /// <param name="desc">Type descriptor.</param>
+        /// <returns>Builder.</returns>
+        private PortableBuilderImpl Builder0(PortableBuilderImpl parent, PortableUserObject obj, 
+            IPortableTypeDescriptor desc)
+        {
+            return new PortableBuilderImpl(this, parent, obj, desc);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/SerializableObjectHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/SerializableObjectHolder.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/SerializableObjectHolder.cs
new file mode 100644
index 0000000..a3a9fe7
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/SerializableObjectHolder.cs
@@ -0,0 +1,66 @@
+/*
+ * 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.Portable
+{
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Wraps Serializable item in a portable.
+    /// </summary>
+    internal class SerializableObjectHolder : IPortableWriteAware
+    {
+        /** */
+        private readonly object _item;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="SerializableObjectHolder"/> class.
+        /// </summary>
+        /// <param name="item">The item to wrap.</param>
+        public SerializableObjectHolder(object item)
+        {
+            _item = item;
+        }
+
+        /// <summary>
+        /// Gets the item to wrap.
+        /// </summary>
+        public object Item
+        {
+            get { return _item; }
+        }
+
+        /** <inheritDoc /> */
+        public void WritePortable(IPortableWriter writer)
+        {
+            var writer0 = (PortableWriterImpl)writer.RawWriter();
+
+            writer0.DetachNext();
+
+            PortableUtils.WriteSerializable(writer0, Item);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="SerializableObjectHolder"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        public SerializableObjectHolder(IPortableReader reader)
+        {
+            _item = PortableUtils.ReadSerializable<object>((PortableReaderImpl)reader.RawReader());
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/TypeResolver.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/TypeResolver.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/TypeResolver.cs
new file mode 100644
index 0000000..0785f4a
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/TypeResolver.cs
@@ -0,0 +1,227 @@
+/*
+ * 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.Portable
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Linq;
+    using System.Reflection;
+    using System.Text.RegularExpressions;
+
+    /// <summary>
+    /// Resolves types by name.
+    /// </summary>
+    internal class TypeResolver
+    {
+        /** Regex to parse generic types from portable configuration. Allows nested generics in type arguments. */
+        private static readonly Regex GenericTypeRegex =
+            new Regex(@"([^`,\[\]]*)(?:`[0-9]+)?(?:\[((?:(?<br>\[)|(?<-br>\])|[^\[\]]*)+)\])?", RegexOptions.Compiled);
+
+        /** Assemblies loaded in ReflectionOnly mode. */
+        private readonly Dictionary<string, Assembly> _reflectionOnlyAssemblies = new Dictionary<string, Assembly>();
+
+        /// <summary>
+        /// Resolve type by name.
+        /// </summary>
+        /// <param name="typeName">Name of the type.</param>
+        /// <param name="assemblyName">Optional, name of the assembly.</param>
+        /// <returns>
+        /// Resolved type.
+        /// </returns>
+        public Type ResolveType(string typeName, string assemblyName = null)
+        {
+            Debug.Assert(!string.IsNullOrEmpty(typeName));
+
+            return ResolveType(assemblyName, typeName, AppDomain.CurrentDomain.GetAssemblies())
+                ?? ResolveTypeInReferencedAssemblies(assemblyName, typeName);
+        }
+
+        /// <summary>
+        /// Resolve type by name in specified assembly set.
+        /// </summary>
+        /// <param name="assemblyName">Name of the assembly.</param>
+        /// <param name="typeName">Name of the type.</param>
+        /// <param name="assemblies">Assemblies to look in.</param>
+        /// <returns> 
+        /// Resolved type. 
+        /// </returns>
+        private static Type ResolveType(string assemblyName, string typeName, ICollection<Assembly> assemblies)
+        {
+            return ResolveGenericType(assemblyName, typeName, assemblies) ??
+                   ResolveNonGenericType(assemblyName, typeName, assemblies);
+        }
+
+        /// <summary>
+        /// Resolves non-generic type by searching provided assemblies.
+        /// </summary>
+        /// <param name="assemblyName">Name of the assembly.</param>
+        /// <param name="typeName">Name of the type.</param>
+        /// <param name="assemblies">The assemblies.</param>
+        /// <returns>Resolved type, or null.</returns>
+        private static Type ResolveNonGenericType(string assemblyName, string typeName, ICollection<Assembly> assemblies)
+        {
+            if (!string.IsNullOrEmpty(assemblyName))
+                assemblies = assemblies
+                    .Where(x => x.FullName == assemblyName || x.GetName().Name == assemblyName).ToArray();
+
+            if (!assemblies.Any())
+                return null;
+
+            // Trim assembly qualification
+            var commaIdx = typeName.IndexOf(',');
+
+            if (commaIdx > 0)
+                typeName = typeName.Substring(0, commaIdx);
+
+            return assemblies.Select(a => a.GetType(typeName, false, false)).FirstOrDefault(type => type != null);
+        }
+
+        /// <summary>
+        /// Resolves the name of the generic type by resolving each generic arg separately 
+        /// and substituting it's fully qualified name.
+        /// (Assembly.GetType finds generic types only when arguments are fully qualified).
+        /// </summary>
+        /// <param name="assemblyName">Name of the assembly.</param>
+        /// <param name="typeName">Name of the type.</param>
+        /// <param name="assemblies">Assemblies</param>
+        /// <returns>Fully qualified generic type name, or null if argument(s) could not be resolved.</returns>
+        private static Type ResolveGenericType(string assemblyName, string typeName, ICollection<Assembly> assemblies)
+        {
+            var match = GenericTypeRegex.Match(typeName);
+
+            if (!match.Success || !match.Groups[2].Success)
+                return null;
+
+            // Try to construct generic type; each generic arg can also be a generic type.
+            var genericArgs = GenericTypeRegex.Matches(match.Groups[2].Value)
+                .OfType<Match>().Select(m => m.Value).Where(v => !string.IsNullOrWhiteSpace(v))
+                .Select(v => ResolveType(null, TrimBrackets(v), assemblies)).ToArray();
+
+            if (genericArgs.Any(x => x == null))
+                return null;
+
+            var genericType = ResolveNonGenericType(assemblyName,
+                string.Format("{0}`{1}", match.Groups[1].Value, genericArgs.Length), assemblies);
+
+            if (genericType == null)
+                return null;
+
+            return genericType.MakeGenericType(genericArgs);
+        }
+
+        /// <summary>
+        /// Trims the brackets from generic type arg.
+        /// </summary>
+        private static string TrimBrackets(string s)
+        {
+            return s.StartsWith("[") && s.EndsWith("]") ? s.Substring(1, s.Length - 2) : s;
+        }
+
+        /// <summary>
+        /// Resolve type by name in non-loaded referenced assemblies.
+        /// </summary>
+        /// <param name="assemblyName">Name of the assembly.</param>
+        /// <param name="typeName">Name of the type.</param>
+        /// <returns>
+        /// Resolved type.
+        /// </returns>
+        private Type ResolveTypeInReferencedAssemblies(string assemblyName, string typeName)
+        {
+            ResolveEventHandler resolver = (sender, args) => GetReflectionOnlyAssembly(args.Name);
+
+            AppDomain.CurrentDomain.ReflectionOnlyAssemblyResolve += resolver;
+
+            try
+            {
+                var result = ResolveType(assemblyName, typeName, GetNotLoadedReferencedAssemblies().ToArray());
+
+                if (result == null)
+                    return null;
+
+                // result is from ReflectionOnly assembly, load it properly into current domain
+                var asm = AppDomain.CurrentDomain.Load(result.Assembly.GetName());
+
+                return asm.GetType(result.FullName);
+            }
+            finally
+            {
+                AppDomain.CurrentDomain.ReflectionOnlyAssemblyResolve -= resolver;
+            }
+        }
+
+        /// <summary>
+        /// Gets the reflection only assembly.
+        /// </summary>
+        [SuppressMessage("Microsoft.Design", "CA1031:DoNotCatchGeneralExceptionTypes")]
+        private Assembly GetReflectionOnlyAssembly(string fullName)
+        {
+            Assembly result;
+
+            if (!_reflectionOnlyAssemblies.TryGetValue(fullName, out result))
+            {
+                try
+                {
+                    result = Assembly.ReflectionOnlyLoad(fullName);
+                }
+                catch (Exception)
+                {
+                    // Some assemblies may fail to load
+                    result = null;
+                }
+
+                _reflectionOnlyAssemblies[fullName] = result;
+            }
+
+            return result;
+        }
+
+        /// <summary>
+        /// Recursively gets all referenced assemblies for current app domain, excluding those that are loaded.
+        /// </summary>
+        private IEnumerable<Assembly> GetNotLoadedReferencedAssemblies()
+        {
+            var roots = new Stack<Assembly>(AppDomain.CurrentDomain.GetAssemblies());
+
+            var visited = new HashSet<string>();
+
+            var loaded = new HashSet<string>(roots.Select(x => x.FullName));
+
+            while (roots.Any())
+            {
+                var asm = roots.Pop();
+
+                if (visited.Contains(asm.FullName))
+                    continue;
+
+                if (!loaded.Contains(asm.FullName))
+                    yield return asm;
+
+                visited.Add(asm.FullName);
+
+                foreach (var refAsm in asm.GetReferencedAssemblies()
+                    .Where(x => !visited.Contains(x.FullName))
+                    .Where(x => !loaded.Contains(x.FullName))
+                    .Select(x => GetReflectionOnlyAssembly(x.FullName))
+                    .Where(x => x != null))
+                    roots.Push(refAsm);
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Resource/IResourceInjector.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Resource/IResourceInjector.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Resource/IResourceInjector.cs
new file mode 100644
index 0000000..b751680
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Resource/IResourceInjector.cs
@@ -0,0 +1,27 @@
+/*
+ * 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.Resource
+{
+    /// <summary>
+    /// Resource injector interface.
+    /// </summary>
+    internal interface IResourceInjector
+    {
+        void Inject(object target, object val);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Resource/ResourceFieldInjector.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Resource/ResourceFieldInjector.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Resource/ResourceFieldInjector.cs
new file mode 100644
index 0000000..d48db1f
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Resource/ResourceFieldInjector.cs
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Impl.Resource
+{
+    using System;
+    using System.Reflection;
+    using Apache.Ignite.Core.Impl.Common;
+
+    /// <summary>
+    /// Field resource injector.
+    /// </summary>
+    internal class ResourceFieldInjector : IResourceInjector
+    {
+        /** */
+        private readonly Action<object, object> _inject;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="field">Field.</param>
+        public ResourceFieldInjector(FieldInfo field)
+        {
+            _inject = DelegateConverter.CompileFieldSetter(field);
+        }
+
+        /** <inheritDoc /> */
+        public void Inject(object target, object val)
+        {
+            _inject(target, val);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Resource/ResourceMethodInjector.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Resource/ResourceMethodInjector.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Resource/ResourceMethodInjector.cs
new file mode 100644
index 0000000..9a7d9d3
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Resource/ResourceMethodInjector.cs
@@ -0,0 +1,48 @@
+/*
+ * 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.Resource
+{
+    using System;
+    using System.Reflection;
+    using Apache.Ignite.Core.Impl.Common;
+
+    /// <summary>
+    /// Method resource injector.
+    /// </summary>
+    internal class ResourceMethodInjector : IResourceInjector
+    {
+        /** */
+        private readonly Action<object, object> _inject;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="mthd">Method.</param>
+        public ResourceMethodInjector(MethodInfo mthd)
+        {
+            _inject = DelegateConverter.CompileFunc<Action<object, object>>(mthd.DeclaringType, mthd,
+                new[] {mthd.GetParameters()[0].ParameterType}, new[] {true, false});
+        }
+
+        /** <inheritDoc /> */
+        public void Inject(object target, object val)
+        {
+            _inject(target, val);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Resource/ResourceProcessor.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Resource/ResourceProcessor.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Resource/ResourceProcessor.cs
new file mode 100644
index 0000000..0a41d8c
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Resource/ResourceProcessor.cs
@@ -0,0 +1,105 @@
+/*
+ * 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.Resource
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Cache.Store;
+
+    /// <summary>
+    /// Resource processor.
+    /// </summary>
+    internal class ResourceProcessor
+    {
+        /** Mutex. */
+        private static readonly object Mux = new object();
+        
+        /** Cached descriptors. */
+        private static volatile IDictionary<Type, ResourceTypeDescriptor> _descs = 
+            new Dictionary<Type, ResourceTypeDescriptor>();
+
+        /// <summary>
+        /// Get descriptor for the given type.
+        /// </summary>
+        /// <param name="type">Type.</param>
+        /// <returns></returns>
+        public static ResourceTypeDescriptor Descriptor(Type type)
+        {
+            IDictionary<Type, ResourceTypeDescriptor> descs0 = _descs;
+
+            ResourceTypeDescriptor desc;
+
+            if (!descs0.TryGetValue(type, out desc))
+            {
+                lock (Mux)
+                {
+                    if (!_descs.TryGetValue(type, out desc))
+                    {
+                        // Create descriptor from scratch.
+                        desc = new ResourceTypeDescriptor(type);
+
+                        descs0 = new Dictionary<Type, ResourceTypeDescriptor>(_descs);
+
+                        descs0[type] = desc;
+
+                        _descs = descs0;
+                    }
+                }
+            }
+
+            return desc;
+        }
+
+        /// <summary>
+        /// Inject resources to the given target.
+        /// </summary>
+        /// <param name="target">Target object.</param>
+        /// <param name="grid">Grid.</param>
+        public static void Inject(object target, Ignite grid)
+        {
+            Inject(target, grid.Proxy);
+        }
+
+        /// <summary>
+        /// Inject resources to the given target.
+        /// </summary>
+        /// <param name="target">Target object.</param>
+        /// <param name="grid">Grid.</param>
+        public static void Inject(object target, IgniteProxy grid)
+        {
+            if (target != null) {
+                var desc = Descriptor(target.GetType());
+    
+                desc.InjectIgnite(target, grid);
+            }
+        }
+
+        /// <summary>
+        /// Inject cache store session.
+        /// </summary>
+        /// <param name="store">Store.</param>
+        /// <param name="ses">Store session.</param>
+        public static void InjectStoreSession(ICacheStore store, ICacheStoreSession ses)
+        {
+            Debug.Assert(store != null);
+
+            Descriptor(store.GetType()).InjectStoreSession(store, ses);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Resource/ResourcePropertyInjector.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Resource/ResourcePropertyInjector.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Resource/ResourcePropertyInjector.cs
new file mode 100644
index 0000000..05e2c2d
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Resource/ResourcePropertyInjector.cs
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Impl.Resource
+{
+    using System;
+    using System.Reflection;
+    using Apache.Ignite.Core.Impl.Common;
+
+    /// <summary>
+    /// Property resource injector.
+    /// </summary>
+    internal class ResourcePropertyInjector : IResourceInjector
+    {
+        /** */
+        private readonly Action<object, object> _inject;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="prop">Property.</param>
+        public ResourcePropertyInjector(PropertyInfo prop)
+        {
+            _inject = DelegateConverter.CompilePropertySetter(prop);
+        }
+
+        /** <inheritDoc /> */
+        public void Inject(object target, object val)
+        {
+            _inject(target, val);
+        }
+    }
+}


[22/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableSystemHandlers.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableSystemHandlers.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableSystemHandlers.cs
new file mode 100644
index 0000000..95a6ef8
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableSystemHandlers.cs
@@ -0,0 +1,1336 @@
+/*
+ * 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.Portable
+{
+    using System;
+    using System.Collections;
+    using System.Collections.Concurrent;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+
+    /// <summary>
+    /// Write delegate.
+    /// </summary>
+    /// <param name="writer">Write context.</param>
+    /// <param name="obj">Object to write.</param>
+    internal delegate void PortableSystemWriteDelegate(PortableWriterImpl writer, object obj);
+
+    /// <summary>
+    /// Typed write delegate.
+    /// </summary>
+    /// <param name="stream">Stream.</param>
+    /// <param name="obj">Object to write.</param>
+    // ReSharper disable once TypeParameterCanBeVariant
+    // Generic variance in a delegate causes performance hit
+    internal delegate void PortableSystemTypedWriteDelegate<T>(IPortableStream stream, T obj);
+
+    /**
+     * <summary>Collection of predefined handlers for various system types.</summary>
+     */
+    internal static class PortableSystemHandlers
+    {
+        /** Write handlers. */
+        private static readonly Dictionary<Type, PortableSystemWriteDelegate> WriteHandlers =
+            new Dictionary<Type, PortableSystemWriteDelegate>();
+
+        /** Read handlers. */
+        private static readonly IPortableSystemReader[] ReadHandlers = new IPortableSystemReader[255];
+
+        /** Typed write handler: boolean. */
+        public static readonly PortableSystemTypedWriteDelegate<bool> WriteHndBoolTyped = WriteBoolTyped;
+
+        /** Typed write handler: byte. */
+        public static readonly PortableSystemTypedWriteDelegate<byte> WriteHndByteTyped = WriteByteTyped;
+
+        /** Typed write handler: short. */
+        public static readonly PortableSystemTypedWriteDelegate<short> WriteHndShortTyped = WriteShortTyped;
+
+        /** Typed write handler: char. */
+        public static readonly PortableSystemTypedWriteDelegate<char> WriteHndCharTyped = WriteCharTyped;
+
+        /** Typed write handler: int. */
+        public static readonly PortableSystemTypedWriteDelegate<int> WriteHndIntTyped = WriteIntTyped;
+
+        /** Typed write handler: long. */
+        public static readonly PortableSystemTypedWriteDelegate<long> WriteHndLongTyped = WriteLongTyped;
+
+        /** Typed write handler: float. */
+        public static readonly PortableSystemTypedWriteDelegate<float> WriteHndFloatTyped = WriteFloatTyped;
+
+        /** Typed write handler: double. */
+        public static readonly PortableSystemTypedWriteDelegate<double> WriteHndDoubleTyped = WriteDoubleTyped;
+
+        /** Typed write handler: decimal. */
+        public static readonly PortableSystemTypedWriteDelegate<decimal> WriteHndDecimalTyped = WriteDecimalTyped;
+
+        /** Typed write handler: Date. */
+        public static readonly PortableSystemTypedWriteDelegate<DateTime?> WriteHndDateTyped = WriteDateTyped;
+
+        /** Typed write handler: string. */
+        public static readonly PortableSystemTypedWriteDelegate<string> WriteHndStringTyped = WriteStringTyped;
+
+        /** Typed write handler: Guid. */
+        public static readonly PortableSystemTypedWriteDelegate<Guid?> WriteHndGuidTyped = WriteGuidTyped;
+
+        /** Typed write handler: Portable. */
+        public static readonly PortableSystemTypedWriteDelegate<PortableUserObject> WriteHndPortableTyped = WritePortableTyped;
+
+        /** Typed write handler: boolean array. */
+        public static readonly PortableSystemTypedWriteDelegate<bool[]> WriteHndBoolArrayTyped = WriteBoolArrayTyped;
+
+        /** Typed write handler: byte array. */
+        public static readonly PortableSystemTypedWriteDelegate<byte[]> WriteHndByteArrayTyped = WriteByteArrayTyped;
+
+        /** Typed write handler: short array. */
+        public static readonly PortableSystemTypedWriteDelegate<short[]> WriteHndShortArrayTyped = WriteShortArrayTyped;
+
+        /** Typed write handler: char array. */
+        public static readonly PortableSystemTypedWriteDelegate<char[]> WriteHndCharArrayTyped = WriteCharArrayTyped;
+
+        /** Typed write handler: int array. */
+        public static readonly PortableSystemTypedWriteDelegate<int[]> WriteHndIntArrayTyped = WriteIntArrayTyped;
+
+        /** Typed write handler: long array. */
+        public static readonly PortableSystemTypedWriteDelegate<long[]> WriteHndLongArrayTyped = WriteLongArrayTyped;
+
+        /** Typed write handler: float array. */
+        public static readonly PortableSystemTypedWriteDelegate<float[]> WriteHndFloatArrayTyped = WriteFloatArrayTyped;
+
+        /** Typed write handler: double array. */
+        public static readonly PortableSystemTypedWriteDelegate<double[]> WriteHndDoubleArrayTyped = WriteDoubleArrayTyped;
+
+        /** Typed write handler: decimal array. */
+        public static readonly PortableSystemTypedWriteDelegate<decimal[]> WriteHndDecimalArrayTyped = WriteDecimalArrayTyped;
+
+        /** Typed write handler: Date array. */
+        public static readonly PortableSystemTypedWriteDelegate<DateTime?[]> WriteHndDateArrayTyped = WriteDateArrayTyped;
+
+        /** Typed write handler: string array. */
+        public static readonly PortableSystemTypedWriteDelegate<string[]> WriteHndStringArrayTyped = WriteStringArrayTyped;
+
+        /** Typed write handler: Guid array. */
+        public static readonly PortableSystemTypedWriteDelegate<Guid?[]> WriteHndGuidArrayTyped = WriteGuidArrayTyped;
+
+        /** Write handler: boolean. */
+        public static readonly PortableSystemWriteDelegate WriteHndBool = WriteBool;
+
+        /** Write handler: sbyte. */
+        public static readonly PortableSystemWriteDelegate WriteHndSbyte = WriteSbyte;
+
+        /** Write handler: byte. */
+        public static readonly PortableSystemWriteDelegate WriteHndByte = WriteByte;
+
+        /** Write handler: short. */
+        public static readonly PortableSystemWriteDelegate WriteHndShort = WriteShort;
+
+        /** Write handler: ushort. */
+        public static readonly PortableSystemWriteDelegate WriteHndUshort = WriteUshort;
+
+        /** Write handler: char. */
+        public static readonly PortableSystemWriteDelegate WriteHndChar = WriteChar;
+
+        /** Write handler: int. */
+        public static readonly PortableSystemWriteDelegate WriteHndInt = WriteInt;
+
+        /** Write handler: uint. */
+        public static readonly PortableSystemWriteDelegate WriteHndUint = WriteUint;
+
+        /** Write handler: long. */
+        public static readonly PortableSystemWriteDelegate WriteHndLong = WriteLong;
+
+        /** Write handler: ulong. */
+        public static readonly PortableSystemWriteDelegate WriteHndUlong = WriteUlong;
+
+        /** Write handler: float. */
+        public static readonly PortableSystemWriteDelegate WriteHndFloat = WriteFloat;
+
+        /** Write handler: double. */
+        public static readonly PortableSystemWriteDelegate WriteHndDouble = WriteDouble;
+
+        /** Write handler: decimal. */
+        public static readonly PortableSystemWriteDelegate WriteHndDecimal = WriteDecimal;
+
+        /** Write handler: Date. */
+        public static readonly PortableSystemWriteDelegate WriteHndDate = WriteDate;
+
+        /** Write handler: string. */
+        public static readonly PortableSystemWriteDelegate WriteHndString = WriteString;
+
+        /** Write handler: Guid. */
+        public static readonly PortableSystemWriteDelegate WriteHndGuid = WriteGuid;
+
+        /** Write handler: Portable. */
+        public static readonly PortableSystemWriteDelegate WriteHndPortable = WritePortable;
+
+        /** Write handler: Enum. */
+        public static readonly PortableSystemWriteDelegate WriteHndEnum = WriteEnum;
+
+        /** Write handler: boolean array. */
+        public static readonly PortableSystemWriteDelegate WriteHndBoolArray = WriteBoolArray;
+
+        /** Write handler: sbyte array. */
+        public static readonly PortableSystemWriteDelegate WriteHndSbyteArray = WriteSbyteArray;
+
+        /** Write handler: byte array. */
+        public static readonly PortableSystemWriteDelegate WriteHndByteArray = WriteByteArray;
+
+        /** Write handler: short array. */
+        public static readonly PortableSystemWriteDelegate WriteHndShortArray = WriteShortArray;
+
+        /** Write handler: ushort array. */
+        public static readonly PortableSystemWriteDelegate WriteHndUshortArray = WriteUshortArray;
+
+        /** Write handler: char array. */
+        public static readonly PortableSystemWriteDelegate WriteHndCharArray = WriteCharArray;
+
+        /** Write handler: int array. */
+        public static readonly PortableSystemWriteDelegate WriteHndIntArray = WriteIntArray;
+
+        /** Write handler: uint array. */
+        public static readonly PortableSystemWriteDelegate WriteHndUintArray = WriteUintArray;
+
+        /** Write handler: long array. */
+        public static readonly PortableSystemWriteDelegate WriteHndLongArray = WriteLongArray;
+
+        /** Write handler: ulong array. */
+        public static readonly PortableSystemWriteDelegate WriteHndUlongArray = WriteUlongArray;
+
+        /** Write handler: float array. */
+        public static readonly PortableSystemWriteDelegate WriteHndFloatArray = WriteFloatArray;
+
+        /** Write handler: double array. */
+        public static readonly PortableSystemWriteDelegate WriteHndDoubleArray = WriteDoubleArray;
+
+        /** Write handler: decimal array. */
+        public static readonly PortableSystemWriteDelegate WriteHndDecimalArray = WriteDecimalArray;
+
+        /** Write handler: date array. */
+        public static readonly PortableSystemWriteDelegate WriteHndDateArray = WriteDateArray;
+
+        /** Write handler: string array. */
+        public static readonly PortableSystemWriteDelegate WriteHndStringArray = WriteStringArray;
+
+        /** Write handler: Guid array. */
+        public static readonly PortableSystemWriteDelegate WriteHndGuidArray = WriteGuidArray;
+
+        /** Write handler: Enum array. */
+        public static readonly PortableSystemWriteDelegate WriteHndEnumArray = WriteEnumArray;
+
+        /** Write handler: object array. */
+        public static readonly PortableSystemWriteDelegate WriteHndArray = WriteArray;
+
+        /** Write handler: collection. */
+        public static readonly PortableSystemWriteDelegate WriteHndCollection = WriteCollection;
+
+        /** Write handler: dictionary. */
+        public static readonly PortableSystemWriteDelegate WriteHndDictionary = WriteDictionary;
+
+        /** Write handler: generic collection. */
+        public static readonly PortableSystemWriteDelegate WriteHndGenericCollection =
+            WriteGenericCollection;
+
+        /** Write handler: generic dictionary. */
+        public static readonly PortableSystemWriteDelegate WriteHndGenericDictionary =
+            WriteGenericDictionary;
+
+        /**
+         * <summary>Static initializer.</summary>
+         */
+        static PortableSystemHandlers()
+        {
+            // 1. Primitives.
+
+            ReadHandlers[PortableUtils.TypeBool] = new PortableSystemReader<bool>(s => s.ReadBool());
+
+            WriteHandlers[typeof(sbyte)] = WriteHndSbyte;
+            ReadHandlers[PortableUtils.TypeByte] = new PortableSystemReader<byte>(s => s.ReadByte());
+
+            WriteHandlers[typeof(ushort)] = WriteHndUshort;
+            ReadHandlers[PortableUtils.TypeShort] = new PortableSystemReader<short>(s => s.ReadShort());
+
+            ReadHandlers[PortableUtils.TypeChar] = new PortableSystemReader<char>(s => s.ReadChar());
+
+            WriteHandlers[typeof(uint)] = WriteHndUint;
+            ReadHandlers[PortableUtils.TypeInt] = new PortableSystemReader<int>(s => s.ReadInt());
+
+            WriteHandlers[typeof(ulong)] = WriteHndUlong;
+            ReadHandlers[PortableUtils.TypeLong] = new PortableSystemReader<long>(s => s.ReadLong());
+
+            ReadHandlers[PortableUtils.TypeFloat] = new PortableSystemReader<float>(s => s.ReadFloat());
+
+            ReadHandlers[PortableUtils.TypeDouble] = new PortableSystemReader<double>(s => s.ReadDouble());
+
+            ReadHandlers[PortableUtils.TypeDecimal] = new PortableSystemReader<decimal>(PortableUtils.ReadDecimal);
+
+            // 2. Date.
+            ReadHandlers[PortableUtils.TypeDate] =
+                new PortableSystemReader<DateTime?>(s => PortableUtils.ReadDate(s, false));
+
+            // 3. String.
+            ReadHandlers[PortableUtils.TypeString] = new PortableSystemReader<string>(PortableUtils.ReadString);
+
+            // 4. Guid.
+            ReadHandlers[PortableUtils.TypeGuid] = new PortableSystemReader<Guid?>(PortableUtils.ReadGuid);
+
+            // 5. Primitive arrays.
+            ReadHandlers[PortableUtils.TypeArrayBool] = new PortableSystemReader<bool[]>(PortableUtils.ReadBooleanArray);
+
+            WriteHandlers[typeof(sbyte[])] = WriteHndSbyteArray;
+            ReadHandlers[PortableUtils.TypeArrayByte] =
+                new PortableSystemDualReader<byte[], sbyte[]>(PortableUtils.ReadByteArray, PortableUtils.ReadSbyteArray);
+
+            WriteHandlers[typeof(ushort[])] = WriteHndUshortArray;
+            ReadHandlers[PortableUtils.TypeArrayShort] =
+                new PortableSystemDualReader<short[], ushort[]>(PortableUtils.ReadShortArray,
+                    PortableUtils.ReadUshortArray);
+
+            ReadHandlers[PortableUtils.TypeArrayChar] = 
+                new PortableSystemReader<char[]>(PortableUtils.ReadCharArray);
+
+            WriteHandlers[typeof(uint[])] = WriteHndUintArray;
+            ReadHandlers[PortableUtils.TypeArrayInt] =
+                new PortableSystemDualReader<int[], uint[]>(PortableUtils.ReadIntArray, PortableUtils.ReadUintArray);
+
+
+            WriteHandlers[typeof(ulong[])] = WriteHndUlongArray;
+            ReadHandlers[PortableUtils.TypeArrayLong] =
+                new PortableSystemDualReader<long[], ulong[]>(PortableUtils.ReadLongArray, 
+                    PortableUtils.ReadUlongArray);
+
+            ReadHandlers[PortableUtils.TypeArrayFloat] =
+                new PortableSystemReader<float[]>(PortableUtils.ReadFloatArray);
+
+            ReadHandlers[PortableUtils.TypeArrayDouble] =
+                new PortableSystemReader<double[]>(PortableUtils.ReadDoubleArray);
+
+            ReadHandlers[PortableUtils.TypeArrayDecimal] =
+                new PortableSystemReader<decimal[]>(PortableUtils.ReadDecimalArray);
+
+            // 6. Date array.
+            ReadHandlers[PortableUtils.TypeArrayDate] =
+                new PortableSystemReader<DateTime?[]>(s => PortableUtils.ReadDateArray(s, false));
+
+            // 7. String array.
+            ReadHandlers[PortableUtils.TypeArrayString] = new PortableSystemGenericArrayReader<string>();
+
+            // 8. Guid array.
+            ReadHandlers[PortableUtils.TypeArrayGuid] = new PortableSystemGenericArrayReader<Guid?>();
+
+            // 9. Array.
+            ReadHandlers[PortableUtils.TypeArray] = new PortableSystemReader(ReadArray);
+
+            // 10. Predefined collections.
+            WriteHandlers[typeof(ArrayList)] = WriteArrayList;
+
+            // 11. Predefined dictionaries.
+            WriteHandlers[typeof(Hashtable)] = WriteHashtable;
+
+            // 12. Arbitrary collection.
+            ReadHandlers[PortableUtils.TypeCollection] = new PortableSystemReader(ReadCollection);
+
+            // 13. Arbitrary dictionary.
+            ReadHandlers[PortableUtils.TypeDictionary] = new PortableSystemReader(ReadDictionary);
+
+            // 14. Map entry.
+            WriteHandlers[typeof(DictionaryEntry)] = WriteMapEntry;
+            ReadHandlers[PortableUtils.TypeMapEntry] = new PortableSystemReader(ReadMapEntry);
+
+            // 15. Portable.
+            WriteHandlers[typeof(PortableUserObject)] = WritePortable;
+
+            // 16. Enum.
+            ReadHandlers[PortableUtils.TypeEnum] = new PortableSystemReader<int>(PortableUtils.ReadEnum<int>);
+            ReadHandlers[PortableUtils.TypeArrayEnum] = new PortableSystemReader(ReadEnumArray);
+        }
+
+        /**
+         * <summary>Get write handler for type.</summary>
+         * <param name="type">Type.</param>
+         * <returns>Handler or null if cannot be hanled in special way.</returns>
+         */
+        public static PortableSystemWriteDelegate WriteHandler(Type type)
+        {
+            PortableSystemWriteDelegate handler;
+
+            if (WriteHandlers.TryGetValue(type, out handler))
+                return handler;
+
+            // 1. Array?
+            if (type.IsArray)
+            {
+                if (type.GetElementType().IsEnum)
+                    return WriteEnumArray;
+                return WriteArray;
+            }
+
+            // 2. Enum?
+            if (type.IsEnum)
+                return WriteEnum;
+
+            // 3. Collection?
+            PortableCollectionInfo info = PortableCollectionInfo.Info(type);
+
+            if (info.IsAny)
+                return info.WriteHandler;
+
+            // No special handler found.
+            return null;
+        }
+
+        /// <summary>
+        /// Reads an object of predefined type.
+        /// </summary>
+        public static T ReadSystemType<T>(byte typeId, PortableReaderImpl ctx)
+        {
+            var handler = ReadHandlers[typeId];
+
+            Debug.Assert(handler != null, "Cannot find predefined read handler: " + typeId);
+            
+            return handler.Read<T>(ctx);
+        }
+
+        /**
+         * <summary>Write boolean.</summary>
+         */
+        private static void WriteBool(PortableWriterImpl ctx, object obj)
+        {
+            WriteBoolTyped(ctx.Stream, (bool)obj);
+        }
+
+        /**
+         * <summary>Write boolean.</summary>
+         */
+        private static void WriteBoolTyped(IPortableStream stream, bool obj)
+        {
+            stream.WriteByte(PortableUtils.TypeBool);
+
+            stream.WriteBool(obj);
+        }
+
+        /**
+         * <summary>Write sbyte.</summary>
+         */
+        private static unsafe void WriteSbyte(PortableWriterImpl ctx, object obj)
+        {
+            sbyte val = (sbyte)obj;
+
+            ctx.Stream.WriteByte(PortableUtils.TypeByte);
+            ctx.Stream.WriteByte(*(byte*)&val);
+        }
+
+        /**
+         * <summary>Write byte.</summary>
+         */
+        private static void WriteByte(PortableWriterImpl ctx, object obj)
+        {
+            WriteByteTyped(ctx.Stream, (byte)obj);
+        }
+
+        /**
+         * <summary>Write byte.</summary>
+         */
+        private static void WriteByteTyped(IPortableStream stream, byte obj)
+        {
+            stream.WriteByte(PortableUtils.TypeByte);
+            stream.WriteByte(obj);
+        }
+
+        /**
+         * <summary>Write short.</summary>
+         */
+        private static void WriteShort(PortableWriterImpl ctx, object obj)
+        {
+            WriteShortTyped(ctx.Stream, (short)obj);
+        }
+
+        /**
+         * <summary>Write short.</summary>
+         */
+        private static void WriteShortTyped(IPortableStream stream, short obj)
+        {
+            stream.WriteByte(PortableUtils.TypeShort);
+
+            stream.WriteShort(obj);
+        }
+
+        /**
+         * <summary>Write ushort.</summary>
+         */
+        private static unsafe void WriteUshort(PortableWriterImpl ctx, object obj)
+        {
+            ushort val = (ushort)obj;
+
+            ctx.Stream.WriteByte(PortableUtils.TypeShort);
+
+            ctx.Stream.WriteShort(*(short*)&val);
+        }
+
+        /**
+         * <summary>Write char.</summary>
+         */
+        private static void WriteChar(PortableWriterImpl ctx, object obj)
+        {
+            WriteCharTyped(ctx.Stream, (char)obj);
+        }
+
+        /**
+         * <summary>Write char.</summary>
+         */
+        private static void WriteCharTyped(IPortableStream stream, char obj)
+        {
+            stream.WriteByte(PortableUtils.TypeChar);
+
+            stream.WriteChar(obj);
+        }
+
+        /**
+         * <summary>Write int.</summary>
+         */
+        private static void WriteInt(PortableWriterImpl ctx, object obj)
+        {
+            WriteIntTyped(ctx.Stream, (int)obj);
+        }
+
+        /**
+         * <summary>Write int.</summary>
+         */
+        private static void WriteIntTyped(IPortableStream stream, int obj)
+        {
+            stream.WriteByte(PortableUtils.TypeInt);
+            stream.WriteInt(obj);
+        }
+
+        /**
+         * <summary>Write uint.</summary>
+         */
+        private static unsafe void WriteUint(PortableWriterImpl ctx, object obj)
+        {
+            uint val = (uint)obj;
+
+            ctx.Stream.WriteByte(PortableUtils.TypeInt);
+            ctx.Stream.WriteInt(*(int*)&val);
+        }
+
+        /**
+         * <summary>Write long.</summary>
+         */
+        private static void WriteLong(PortableWriterImpl ctx, object obj)
+        {
+            WriteLongTyped(ctx.Stream, (long)obj);
+        }
+
+        /**
+         * <summary>Write long.</summary>
+         */
+        private static void WriteLongTyped(IPortableStream stream, long obj)
+        {
+            stream.WriteByte(PortableUtils.TypeLong);
+            stream.WriteLong(obj);
+        }
+
+        /**
+         * <summary>Write ulong.</summary>
+         */
+        private static unsafe void WriteUlong(PortableWriterImpl ctx, object obj)
+        {
+            ulong val = (ulong)obj;
+
+            ctx.Stream.WriteByte(PortableUtils.TypeLong);
+            ctx.Stream.WriteLong(*(long*)&val);
+        }
+
+        /**
+         * <summary>Write float.</summary>
+         */
+        private static void WriteFloat(PortableWriterImpl ctx, object obj)
+        {
+            WriteFloatTyped(ctx.Stream, (float)obj);
+        }
+
+        /**
+         * <summary>Write float.</summary>
+         */
+        private static void WriteFloatTyped(IPortableStream stream, float obj)
+        {
+            stream.WriteByte(PortableUtils.TypeFloat);
+            stream.WriteFloat(obj);
+        }
+
+        /**
+         * <summary>Write double.</summary>
+         */
+        private static void WriteDouble(PortableWriterImpl ctx, object obj)
+        {
+            WriteDoubleTyped(ctx.Stream, (double)obj);
+        }
+
+        /**
+         * <summary>Write double.</summary>
+         */
+        private static void WriteDoubleTyped(IPortableStream stream, double obj)
+        {
+            stream.WriteByte(PortableUtils.TypeDouble);
+            stream.WriteDouble(obj);
+        }
+
+        /**
+         * <summary>Write decimal.</summary>
+         */
+        private static void WriteDecimal(PortableWriterImpl ctx, object obj)
+        {
+            WriteDecimalTyped(ctx.Stream, (decimal)obj);
+        }
+
+        /**
+         * <summary>Write double.</summary>
+         */
+        private static void WriteDecimalTyped(IPortableStream stream, decimal obj)
+        {
+            stream.WriteByte(PortableUtils.TypeDecimal);
+
+            PortableUtils.WriteDecimal(obj, stream);
+        }
+
+        /**
+         * <summary>Write date.</summary>
+         */
+        private static void WriteDate(PortableWriterImpl ctx, object obj)
+        {
+            WriteDateTyped(ctx.Stream, (DateTime?)obj);
+        }
+
+        /**
+         * <summary>Write double.</summary>
+         */
+        private static void WriteDateTyped(IPortableStream stream, DateTime? obj)
+        {
+            stream.WriteByte(PortableUtils.TypeDate);
+
+            PortableUtils.WriteDate(obj, stream);
+        }
+
+        /**
+         * <summary>Write string.</summary>
+         */
+        private static void WriteString(PortableWriterImpl ctx, object obj)
+        {
+            WriteStringTyped(ctx.Stream, (string)obj);
+        }
+
+        /**
+         * <summary>Write string.</summary>
+         */
+        private static void WriteStringTyped(IPortableStream stream, string obj)
+        {
+            stream.WriteByte(PortableUtils.TypeString);
+
+            PortableUtils.WriteString(obj, stream);
+        }
+
+        /**
+         * <summary>Write Guid.</summary>
+         */
+        private static void WriteGuid(PortableWriterImpl ctx, object obj)
+        {
+            WriteGuidTyped(ctx.Stream, (Guid?)obj);
+        }
+
+        /**
+         * <summary>Write Guid.</summary>
+         */
+        private static void WriteGuidTyped(IPortableStream stream, Guid? obj)
+        {
+            stream.WriteByte(PortableUtils.TypeGuid);
+
+            PortableUtils.WriteGuid(obj, stream);
+        }
+
+        /**
+         * <summary>Write bool array.</summary>
+         */
+        private static void WriteBoolArray(PortableWriterImpl ctx, object obj)
+        {
+            WriteBoolArrayTyped(ctx.Stream, (bool[])obj);
+        }
+
+        /**
+         * <summary>Write bool array.</summary>
+         */
+        private static void WriteBoolArrayTyped(IPortableStream stream, bool[] obj)
+        {
+            stream.WriteByte(PortableUtils.TypeArrayBool);
+
+            PortableUtils.WriteBooleanArray(obj, stream);
+        }
+
+        /**
+         * <summary>Write byte array.</summary>
+         */
+        private static void WriteByteArray(PortableWriterImpl ctx, object obj)
+        {
+            WriteByteArrayTyped(ctx.Stream, (byte[])obj);
+        }
+
+        /**
+         * <summary>Write byte array.</summary>
+         */
+        private static void WriteByteArrayTyped(IPortableStream stream, byte[] obj)
+        {
+            stream.WriteByte(PortableUtils.TypeArrayByte);
+
+            PortableUtils.WriteByteArray(obj, stream);
+        }
+
+        /**
+         * <summary>Write sbyte array.</summary>
+         */
+        private static void WriteSbyteArray(PortableWriterImpl ctx, object obj)
+        {
+            ctx.Stream.WriteByte(PortableUtils.TypeArrayByte);
+
+            PortableUtils.WriteByteArray((byte[])(Array)obj, ctx.Stream);
+        }
+
+        /**
+         * <summary>Write short array.</summary>
+         */
+        private static void WriteShortArray(PortableWriterImpl ctx, object obj)
+        {
+            WriteShortArrayTyped(ctx.Stream, (short[])obj);
+        }
+
+        /**
+         * <summary>Write short array.</summary>
+         */
+        private static void WriteShortArrayTyped(IPortableStream stream, short[] obj)
+        {
+            stream.WriteByte(PortableUtils.TypeArrayShort);
+
+            PortableUtils.WriteShortArray(obj, stream);
+        }
+
+        /**
+         * <summary>Write ushort array.</summary>
+         */
+        private static void WriteUshortArray(PortableWriterImpl ctx, object obj)
+        {
+            ctx.Stream.WriteByte(PortableUtils.TypeArrayShort);
+
+            PortableUtils.WriteShortArray((short[])(Array)obj, ctx.Stream);
+        }
+
+        /**
+         * <summary>Write char array.</summary>
+         */
+        private static void WriteCharArray(PortableWriterImpl ctx, object obj)
+        {
+            WriteCharArrayTyped(ctx.Stream, (char[])obj);
+        }
+
+        /**
+         * <summary>Write char array.</summary>
+         */
+        private static void WriteCharArrayTyped(IPortableStream stream, char[] obj)
+        {
+            stream.WriteByte(PortableUtils.TypeArrayChar);
+
+            PortableUtils.WriteCharArray(obj, stream);
+        }
+
+        /**
+         * <summary>Write int array.</summary>
+         */
+        private static void WriteIntArray(PortableWriterImpl ctx, object obj)
+        {
+            WriteIntArrayTyped(ctx.Stream, (int[])obj);
+        }
+
+        /**
+         * <summary>Write int array.</summary>
+         */
+        private static void WriteIntArrayTyped(IPortableStream stream, int[] obj)
+        {
+            stream.WriteByte(PortableUtils.TypeArrayInt);
+
+            PortableUtils.WriteIntArray(obj, stream);
+        }
+
+        /**
+         * <summary>Write uint array.</summary>
+         */
+        private static void WriteUintArray(PortableWriterImpl ctx, object obj)
+        {
+            ctx.Stream.WriteByte(PortableUtils.TypeArrayInt);
+
+            PortableUtils.WriteIntArray((int[])(Array)obj, ctx.Stream);
+        }
+
+        /**
+         * <summary>Write long array.</summary>
+         */
+        private static void WriteLongArray(PortableWriterImpl ctx, object obj)
+        {
+            WriteLongArrayTyped(ctx.Stream, (long[])obj);
+        }
+
+        /**
+         * <summary>Write long array.</summary>
+         */
+        private static void WriteLongArrayTyped(IPortableStream stream, long[] obj)
+        {
+            stream.WriteByte(PortableUtils.TypeArrayLong);
+
+            PortableUtils.WriteLongArray(obj, stream);
+        }
+
+        /**
+         * <summary>Write ulong array.</summary>
+         */
+        private static void WriteUlongArray(PortableWriterImpl ctx, object obj)
+        {
+            ctx.Stream.WriteByte(PortableUtils.TypeArrayLong);
+
+            PortableUtils.WriteLongArray((long[])(Array)obj, ctx.Stream);
+        }
+
+        /**
+         * <summary>Write float array.</summary>
+         */
+        private static void WriteFloatArray(PortableWriterImpl ctx, object obj)
+        {
+            WriteFloatArrayTyped(ctx.Stream, (float[])obj);
+        }
+
+        /**
+         * <summary>Write float array.</summary>
+         */
+        private static void WriteFloatArrayTyped(IPortableStream stream, float[] obj)
+        {
+            stream.WriteByte(PortableUtils.TypeArrayFloat);
+
+            PortableUtils.WriteFloatArray(obj, stream);
+        }
+
+        /**
+         * <summary>Write double array.</summary>
+         */
+        private static void WriteDoubleArray(PortableWriterImpl ctx, object obj)
+        {
+            WriteDoubleArrayTyped(ctx.Stream, (double[])obj);
+        }
+
+        /**
+         * <summary>Write double array.</summary>
+         */
+        private static void WriteDoubleArrayTyped(IPortableStream stream, double[] obj)
+        {
+            stream.WriteByte(PortableUtils.TypeArrayDouble);
+
+            PortableUtils.WriteDoubleArray(obj, stream);
+        }
+
+        /**
+         * <summary>Write decimal array.</summary>
+         */
+        private static void WriteDecimalArray(PortableWriterImpl ctx, object obj)
+        {
+            WriteDecimalArrayTyped(ctx.Stream, (decimal[])obj);
+        }
+
+        /**
+         * <summary>Write double array.</summary>
+         */
+        private static void WriteDecimalArrayTyped(IPortableStream stream, decimal[] obj)
+        {
+            stream.WriteByte(PortableUtils.TypeArrayDecimal);
+
+            PortableUtils.WriteDecimalArray(obj, stream);
+        }
+
+        /**
+         * <summary>Write date array.</summary>
+         */
+        private static void WriteDateArray(PortableWriterImpl ctx, object obj)
+        {
+            WriteDateArrayTyped(ctx.Stream, (DateTime?[])obj);
+        }
+
+        /**
+         * <summary>Write date array.</summary>
+         */
+        private static void WriteDateArrayTyped(IPortableStream stream, DateTime?[] obj)
+        {
+            stream.WriteByte(PortableUtils.TypeArrayDate);
+
+            PortableUtils.WriteDateArray(obj, stream);
+        }
+
+        /**
+         * <summary>Write string array.</summary>
+         */
+        private static void WriteStringArray(PortableWriterImpl ctx, object obj)
+        {
+            WriteStringArrayTyped(ctx.Stream, (string[])obj);
+        }
+
+        /**
+         * <summary>Write string array.</summary>
+         */
+        private static void WriteStringArrayTyped(IPortableStream stream, string[] obj)
+        {
+            stream.WriteByte(PortableUtils.TypeArrayString);
+
+            PortableUtils.WriteStringArray(obj, stream);
+        }
+
+        /**
+         * <summary>Write Guid array.</summary>
+         */
+        private static void WriteGuidArray(PortableWriterImpl ctx, object obj)
+        {
+            WriteGuidArrayTyped(ctx.Stream, (Guid?[])obj);
+        }
+
+        /**
+         * <summary>Write Guid array.</summary>
+         */
+        private static void WriteGuidArrayTyped(IPortableStream stream, Guid?[] obj)
+        {
+            stream.WriteByte(PortableUtils.TypeArrayGuid);
+
+            PortableUtils.WriteGuidArray(obj, stream);
+        }
+
+        /**
+         * <summary>Write enum array.</summary>
+         */
+        private static void WriteEnumArray(PortableWriterImpl ctx, object obj)
+        {
+            ctx.Stream.WriteByte(PortableUtils.TypeArrayEnum);
+
+            PortableUtils.WriteArray((Array)obj, ctx, true);
+        }
+
+        /**
+         * <summary>Write array.</summary>
+         */
+        private static void WriteArray(PortableWriterImpl ctx, object obj)
+        {
+            ctx.Stream.WriteByte(PortableUtils.TypeArray);
+
+            PortableUtils.WriteArray((Array)obj, ctx, true);
+        }
+
+        /**
+         * <summary>Write collection.</summary>
+         */
+        private static void WriteCollection(PortableWriterImpl ctx, object obj)
+        {
+            ctx.Stream.WriteByte(PortableUtils.TypeCollection);
+
+            PortableUtils.WriteCollection((ICollection)obj, ctx);
+        }
+
+        /**
+         * <summary>Write generic collection.</summary>
+         */
+        private static void WriteGenericCollection(PortableWriterImpl ctx, object obj)
+        {
+            PortableCollectionInfo info = PortableCollectionInfo.Info(obj.GetType());
+
+            Debug.Assert(info.IsGenericCollection, "Not generic collection: " + obj.GetType().FullName);
+
+            ctx.Stream.WriteByte(PortableUtils.TypeCollection);
+
+            info.WriteGeneric(ctx, obj);
+        }
+
+        /**
+         * <summary>Write dictionary.</summary>
+         */
+        private static void WriteDictionary(PortableWriterImpl ctx, object obj)
+        {
+            ctx.Stream.WriteByte(PortableUtils.TypeDictionary);
+
+            PortableUtils.WriteDictionary((IDictionary)obj, ctx);
+        }
+
+        /**
+         * <summary>Write generic dictionary.</summary>
+         */
+        private static void WriteGenericDictionary(PortableWriterImpl ctx, object obj)
+        {
+            PortableCollectionInfo info = PortableCollectionInfo.Info(obj.GetType());
+
+            Debug.Assert(info.IsGenericDictionary, "Not generic dictionary: " + obj.GetType().FullName);
+
+            ctx.Stream.WriteByte(PortableUtils.TypeDictionary);
+
+            info.WriteGeneric(ctx, obj);
+        }
+
+        /**
+         * <summary>Write ArrayList.</summary>
+         */
+        private static void WriteArrayList(PortableWriterImpl ctx, object obj)
+        {
+            ctx.Stream.WriteByte(PortableUtils.TypeCollection);
+
+            PortableUtils.WriteTypedCollection((ICollection)obj, ctx, PortableUtils.CollectionArrayList);
+        }
+
+        /**
+         * <summary>Write Hashtable.</summary>
+         */
+        private static void WriteHashtable(PortableWriterImpl ctx, object obj)
+        {
+            ctx.Stream.WriteByte(PortableUtils.TypeDictionary);
+
+            PortableUtils.WriteTypedDictionary((IDictionary)obj, ctx, PortableUtils.MapHashMap);
+        }
+
+        /**
+         * <summary>Write map entry.</summary>
+         */
+        private static void WriteMapEntry(PortableWriterImpl ctx, object obj)
+        {
+            ctx.Stream.WriteByte(PortableUtils.TypeMapEntry);
+
+            PortableUtils.WriteMapEntry(ctx, (DictionaryEntry)obj);
+        }
+
+        /**
+         * <summary>Write portable object.</summary>
+         */
+        private static void WritePortable(PortableWriterImpl ctx, object obj)
+        {
+            ctx.Stream.WriteByte(PortableUtils.TypePortable);
+
+            PortableUtils.WritePortable(ctx.Stream, (PortableUserObject)obj);
+        }
+
+        /**
+         * <summary>Write portable object.</summary>
+         */
+        private static void WritePortableTyped(IPortableStream stream, PortableUserObject obj)
+        {
+            stream.WriteByte(PortableUtils.TypePortable);
+
+            PortableUtils.WritePortable(stream, obj);
+        }
+
+        /// <summary>
+        /// Write enum.
+        /// </summary>
+        private static void WriteEnum(PortableWriterImpl ctx, object obj)
+        {
+            ctx.Stream.WriteByte(PortableUtils.TypeEnum);
+
+            PortableUtils.WriteEnum(ctx.Stream, (Enum)obj);
+        }
+
+        /**
+         * <summary>Read enum array.</summary>
+         */
+        private static object ReadEnumArray(PortableReaderImpl ctx, Type type)
+        {
+            return PortableUtils.ReadArray(ctx, true, type.GetElementType());
+        }
+
+        /**
+         * <summary>Read array.</summary>
+         */
+        private static object ReadArray(PortableReaderImpl ctx, Type type)
+        {
+            var elemType = type.IsArray ? type.GetElementType() : typeof(object);
+
+            return PortableUtils.ReadArray(ctx, true, elemType);
+        }
+
+        /**
+         * <summary>Read collection.</summary>
+         */
+        private static object ReadCollection(PortableReaderImpl ctx, Type type)
+        {
+            PortableCollectionInfo info = PortableCollectionInfo.Info(type);
+
+            return info.IsGenericCollection 
+                ? info.ReadGeneric(ctx)
+                : PortableUtils.ReadCollection(ctx, null, null);
+        }
+
+        /**
+         * <summary>Read dictionary.</summary>
+         */
+        private static object ReadDictionary(PortableReaderImpl ctx, Type type)
+        {
+            PortableCollectionInfo info = PortableCollectionInfo.Info(type);
+
+            return info.IsGenericDictionary
+                ? info.ReadGeneric(ctx)
+                : PortableUtils.ReadDictionary(ctx, null);
+        }
+
+        /**
+         * <summary>Read map entry.</summary>
+         */
+        private static object ReadMapEntry(PortableReaderImpl ctx, Type type)
+        {
+            return PortableUtils.ReadMapEntry(ctx);
+        }
+
+        /**
+         * <summary>Create new ArrayList.</summary>
+         * <param name="len">Length.</param>
+         * <returns>ArrayList.</returns>
+         */
+        public static ICollection CreateArrayList(int len)
+        {
+            return new ArrayList(len);
+        }
+
+        /**
+         * <summary>Add element to array list.</summary>
+         * <param name="col">Array list.</param>
+         * <param name="elem">Element.</param>
+         */
+        public static void AddToArrayList(ICollection col, object elem)
+        {
+            ((ArrayList) col).Add(elem);
+        }
+
+        /**
+         * <summary>Create new List.</summary>
+         * <param name="len">Length.</param>
+         * <returns>List.</returns>
+         */
+        public static ICollection<T> CreateList<T>(int len)
+        {
+            return new List<T>(len);
+        }
+
+        /**
+         * <summary>Create new LinkedList.</summary>
+         * <param name="len">Length.</param>
+         * <returns>LinkedList.</returns>
+         */
+        public static ICollection<T> CreateLinkedList<T>(int len)
+        {
+            return new LinkedList<T>();
+        }
+
+        /**
+         * <summary>Create new HashSet.</summary>
+         * <param name="len">Length.</param>
+         * <returns>HashSet.</returns>
+         */
+        public static ICollection<T> CreateHashSet<T>(int len)
+        {
+            return new HashSet<T>();
+        }
+
+        /**
+         * <summary>Create new SortedSet.</summary>
+         * <param name="len">Length.</param>
+         * <returns>SortedSet.</returns>
+         */
+        public static ICollection<T> CreateSortedSet<T>(int len)
+        {
+            return new SortedSet<T>();
+        }
+
+        /**
+         * <summary>Create new Hashtable.</summary>
+         * <param name="len">Length.</param>
+         * <returns>Hashtable.</returns>
+         */
+        public static IDictionary CreateHashtable(int len)
+        {
+            return new Hashtable(len);
+        }
+
+        /**
+         * <summary>Create new Dictionary.</summary>
+         * <param name="len">Length.</param>
+         * <returns>Dictionary.</returns>
+         */
+        public static IDictionary<TK, TV> CreateDictionary<TK, TV>(int len)
+        {
+            return new Dictionary<TK, TV>(len);
+        }
+
+        /**
+         * <summary>Create new SortedDictionary.</summary>
+         * <param name="len">Length.</param>
+         * <returns>SortedDictionary.</returns>
+         */
+        public static IDictionary<TK, TV> CreateSortedDictionary<TK, TV>(int len)
+        {
+            return new SortedDictionary<TK, TV>();
+        }
+
+        /**
+         * <summary>Create new ConcurrentDictionary.</summary>
+         * <param name="len">Length.</param>
+         * <returns>ConcurrentDictionary.</returns>
+         */
+        public static IDictionary<TK, TV> CreateConcurrentDictionary<TK, TV>(int len)
+        {
+            return new ConcurrentDictionary<TK, TV>(Environment.ProcessorCount, len);
+        }
+
+
+        /**
+         * <summary>Read delegate.</summary>
+         * <param name="ctx">Read context.</param>
+         * <param name="type">Type.</param>
+         */
+        private delegate object PortableSystemReadDelegate(PortableReaderImpl ctx, Type type);
+
+        /// <summary>
+        /// System type reader.
+        /// </summary>
+        private interface IPortableSystemReader
+        {
+            /// <summary>
+            /// Reads a value of specified type from reader.
+            /// </summary>
+            T Read<T>(PortableReaderImpl ctx);
+        }
+
+        /// <summary>
+        /// System type generic reader.
+        /// </summary>
+        private interface IPortableSystemReader<out T>
+        {
+            /// <summary>
+            /// Reads a value of specified type from reader.
+            /// </summary>
+            T Read(PortableReaderImpl ctx);
+        }
+
+        /// <summary>
+        /// Default reader with boxing.
+        /// </summary>
+        private class PortableSystemReader : IPortableSystemReader
+        {
+            /** */
+            private readonly PortableSystemReadDelegate _readDelegate;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="PortableSystemReader"/> class.
+            /// </summary>
+            /// <param name="readDelegate">The read delegate.</param>
+            public PortableSystemReader(PortableSystemReadDelegate readDelegate)
+            {
+                Debug.Assert(readDelegate != null);
+
+                _readDelegate = readDelegate;
+            }
+
+            /** <inheritdoc /> */
+            public T Read<T>(PortableReaderImpl ctx)
+            {
+                return (T)_readDelegate(ctx, typeof(T));
+            }
+        }
+
+        /// <summary>
+        /// Reader without boxing.
+        /// </summary>
+        private class PortableSystemReader<T> : IPortableSystemReader
+        {
+            /** */
+            private readonly Func<IPortableStream, T> _readDelegate;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="PortableSystemReader{T}"/> class.
+            /// </summary>
+            /// <param name="readDelegate">The read delegate.</param>
+            public PortableSystemReader(Func<IPortableStream, T> readDelegate)
+            {
+                Debug.Assert(readDelegate != null);
+
+                _readDelegate = readDelegate;
+            }
+
+            /** <inheritdoc /> */
+            public TResult Read<TResult>(PortableReaderImpl ctx)
+            {
+                return TypeCaster<TResult>.Cast(_readDelegate(ctx.Stream));
+            }
+        }
+
+        /// <summary>
+        /// Reader without boxing.
+        /// </summary>
+        private class PortableSystemGenericArrayReader<T> : IPortableSystemReader
+        {
+            public TResult Read<TResult>(PortableReaderImpl ctx)
+            {
+                return TypeCaster<TResult>.Cast(PortableUtils.ReadGenericArray<T>(ctx, false));
+            }
+        }
+
+        /// <summary>
+        /// Reader with selection based on requested type.
+        /// </summary>
+        private class PortableSystemDualReader<T1, T2> : IPortableSystemReader, IPortableSystemReader<T2>
+        {
+            /** */
+            private readonly Func<IPortableStream, T1> _readDelegate1;
+
+            /** */
+            private readonly Func<IPortableStream, T2> _readDelegate2;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="PortableSystemDualReader{T1, T2}"/> class.
+            /// </summary>
+            /// <param name="readDelegate1">The read delegate1.</param>
+            /// <param name="readDelegate2">The read delegate2.</param>
+            public PortableSystemDualReader(Func<IPortableStream, T1> readDelegate1, Func<IPortableStream, T2> readDelegate2)
+            {
+                Debug.Assert(readDelegate1 != null);
+                Debug.Assert(readDelegate2 != null);
+
+                _readDelegate1 = readDelegate1;
+                _readDelegate2 = readDelegate2;
+            }
+
+            /** <inheritdoc /> */
+            T2 IPortableSystemReader<T2>.Read(PortableReaderImpl ctx)
+            {
+                return _readDelegate2(ctx.Stream);
+            }
+
+            /** <inheritdoc /> */
+            public T Read<T>(PortableReaderImpl ctx)
+            {
+                // Can't use "as" because of variance. 
+                // For example, IPortableSystemReader<byte[]> can be cast to IPortableSystemReader<sbyte[]>, which
+                // will cause incorrect behavior.
+                if (typeof (T) == typeof (T2))  
+                    return ((IPortableSystemReader<T>) this).Read(ctx);
+
+                return TypeCaster<T>.Cast(_readDelegate1(ctx.Stream));
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableSystemTypeSerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableSystemTypeSerializer.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableSystemTypeSerializer.cs
new file mode 100644
index 0000000..014955b
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableSystemTypeSerializer.cs
@@ -0,0 +1,62 @@
+/*
+ * 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.Portable
+{
+    using System;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Portable serializer for system types.
+    /// </summary>
+    /// <typeparam name="T">Object type.</typeparam>
+    internal class PortableSystemTypeSerializer<T> : IPortableSystemTypeSerializer where T : IPortableWriteAware
+    {
+        /** Ctor delegate. */
+        private readonly Func<PortableReaderImpl, T> _ctor;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="PortableSystemTypeSerializer{T}"/> class.
+        /// </summary>
+        /// <param name="ctor">Constructor delegate.</param>
+        public PortableSystemTypeSerializer(Func<PortableReaderImpl, T> ctor)
+        {
+            Debug.Assert(ctor != null);
+
+            _ctor = ctor;
+        }
+
+        /** <inheritdoc /> */
+        public void WritePortable(object obj, IPortableWriter writer)
+        {
+            ((T) obj).WritePortable(writer);
+        }
+
+        /** <inheritdoc /> */
+        public void ReadPortable(object obj, IPortableReader reader)
+        {
+            throw new NotSupportedException("System serializer does not support ReadPortable.");
+        }
+
+        /** <inheritdoc /> */
+        public object ReadInstance(PortableReaderImpl reader)
+        {
+            return _ctor(reader);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableUserObject.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableUserObject.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableUserObject.cs
new file mode 100644
index 0000000..3ca8a5f
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableUserObject.cs
@@ -0,0 +1,385 @@
+/*
+ * 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.Portable
+{
+    using System.Collections;
+    using System.Collections.Generic;
+    using System.IO;
+    using System.Runtime.CompilerServices;
+    using System.Text;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// User portable object.
+    /// </summary>
+    internal class PortableUserObject : IPortableObject
+    {
+        /** Marshaller. */
+        private readonly PortableMarshaller _marsh;
+
+        /** Raw data of this portable object. */
+        private readonly byte[] _data;
+
+        /** Offset in data array. */
+        private readonly int _offset;
+
+        /** Type ID. */
+        private readonly int _typeId;
+
+        /** Hash code. */
+        private readonly int _hashCode;
+
+        /** Fields. */
+        private volatile IDictionary<int, int> _fields;
+
+        /** Deserialized value. */
+        private object _deserialized;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="PortableUserObject"/> class.
+        /// </summary>
+        /// <param name="marsh">Marshaller.</param>
+        /// <param name="data">Raw data of this portable object.</param>
+        /// <param name="offset">Offset in data array.</param>
+        /// <param name="typeId">Type ID.</param>
+        /// <param name="hashCode">Hash code.</param>
+        public PortableUserObject(PortableMarshaller marsh, byte[] data, int offset, int typeId, int hashCode)
+        {
+            _marsh = marsh;
+
+            _data = data;
+            _offset = offset;
+
+            _typeId = typeId;
+            _hashCode = hashCode;
+        }
+
+        /** <inheritdoc /> */
+        public int TypeId()
+        {
+            return _typeId;
+        }
+
+        /** <inheritdoc /> */
+        public T Field<T>(string fieldName)
+        {
+            return Field<T>(fieldName, null);
+        }
+
+        /** <inheritdoc /> */
+        public T Deserialize<T>()
+        {
+            return Deserialize<T>(PortableMode.Deserialize);
+        }
+
+        /// <summary>
+        /// Internal deserialization routine.
+        /// </summary>
+        /// <param name="mode">The mode.</param>
+        /// <returns>
+        /// Deserialized object.
+        /// </returns>
+        private T Deserialize<T>(PortableMode mode)
+        {
+            if (_deserialized == null)
+            {
+                IPortableStream stream = new PortableHeapStream(_data);
+
+                stream.Seek(_offset, SeekOrigin.Begin);
+
+                T res = _marsh.Unmarshal<T>(stream, mode);
+
+                IPortableTypeDescriptor desc = _marsh.Descriptor(true, _typeId);
+
+                if (!desc.KeepDeserialized)
+                    return res;
+
+                _deserialized = res;
+            }
+
+            return (T)_deserialized;
+        }
+
+        /** <inheritdoc /> */
+        public IPortableMetadata Metadata()
+        {
+            return _marsh.Metadata(_typeId);
+        }
+
+        /// <summary>
+        /// Raw data of this portable object.
+        /// </summary>
+        public byte[] Data
+        {
+            get { return _data; }
+        }
+
+        /// <summary>
+        /// Offset in data array.
+        /// </summary>
+        public int Offset
+        {
+            get { return _offset; }
+        }
+
+        /// <summary>
+        /// Get field with builder.
+        /// </summary>
+        /// <typeparam name="T"></typeparam>
+        /// <param name="fieldName"></param>
+        /// <param name="builder"></param>
+        /// <returns></returns>
+        public T Field<T>(string fieldName, PortableBuilderImpl builder)
+        {
+            IPortableTypeDescriptor desc = _marsh.Descriptor(true, _typeId);
+
+            InitializeFields();
+
+            int fieldId = PortableUtils.FieldId(_typeId, fieldName, desc.NameConverter, desc.Mapper);
+
+            int pos;
+
+            if (_fields.TryGetValue(fieldId, out pos))
+            {
+                if (builder != null)
+                {
+                    // Read in scope of build process.
+                    T res;
+
+                    if (!builder.CachedField(pos, out res))
+                    {
+                        res = Field0<T>(pos, builder);
+
+                        builder.CacheField(pos, res);
+                    }
+
+                    return res;
+                }
+                return Field0<T>(pos, null);
+            }
+            return default(T);
+        }
+
+        /// <summary>
+        /// Lazy fields initialization routine.
+        /// </summary>
+        private void InitializeFields()
+        {
+            if (_fields == null)
+            {
+                IPortableStream stream = new PortableHeapStream(_data);
+
+                stream.Seek(_offset + 14, SeekOrigin.Begin);
+
+                int rawDataOffset = stream.ReadInt();
+
+                _fields = PortableUtils.ObjectFields(stream, _typeId, rawDataOffset);
+            }
+        }
+
+        /// <summary>
+        /// Gets field value on the given object.
+        /// </summary>
+        /// <param name="pos">Position.</param>
+        /// <param name="builder">Builder.</param>
+        /// <returns>Field value.</returns>
+        private T Field0<T>(int pos, PortableBuilderImpl builder)
+        {
+            IPortableStream stream = new PortableHeapStream(_data);
+
+            stream.Seek(pos, SeekOrigin.Begin);
+
+            return _marsh.Unmarshal<T>(stream, PortableMode.ForcePortable, builder);
+        }
+
+        /** <inheritdoc /> */
+        public override int GetHashCode()
+        {
+            return _hashCode;
+        }
+
+        /** <inheritdoc /> */
+        public override bool Equals(object obj)
+        {
+            if (this == obj)
+                return true;
+
+            PortableUserObject that = obj as PortableUserObject;
+
+            if (that != null)
+            {
+                if (_data == that._data && _offset == that._offset)
+                    return true;
+
+                // 1. Check hash code and type IDs.
+                if (_hashCode == that._hashCode && _typeId == that._typeId)
+                {
+                    // 2. Check if objects have the same field sets.
+                    InitializeFields();
+                    that.InitializeFields();
+
+                    if (_fields.Keys.Count != that._fields.Keys.Count)
+                        return false;
+
+                    foreach (int id in _fields.Keys)
+                    {
+                        if (!that._fields.Keys.Contains(id))
+                            return false;
+                    }
+
+                    // 3. Check if objects have the same field values.
+                    foreach (KeyValuePair<int, int> field in _fields)
+                    {
+                        object fieldVal = Field0<object>(field.Value, null);
+                        object thatFieldVal = that.Field0<object>(that._fields[field.Key], null);
+
+                        if (!Equals(fieldVal, thatFieldVal))
+                            return false;
+                    }
+
+                    // 4. Check if objects have the same raw data.
+                    IPortableStream stream = new PortableHeapStream(_data);
+                    stream.Seek(_offset + 10, SeekOrigin.Begin);
+                    int len = stream.ReadInt();
+                    int rawOffset = stream.ReadInt();
+
+                    IPortableStream thatStream = new PortableHeapStream(that._data);
+                    thatStream.Seek(_offset + 10, SeekOrigin.Begin);
+                    int thatLen = thatStream.ReadInt();
+                    int thatRawOffset = thatStream.ReadInt();
+
+                    return PortableUtils.CompareArrays(_data, _offset + rawOffset, len - rawOffset, that._data,
+                        that._offset + thatRawOffset, thatLen - thatRawOffset);
+                }
+            }
+
+            return false;
+        }
+
+        /** <inheritdoc /> */
+        public override string ToString()
+        {
+            return ToString(new Dictionary<int, int>());            
+        }
+
+        /// <summary>
+        /// ToString implementation.
+        /// </summary>
+        /// <param name="handled">Already handled objects.</param>
+        /// <returns>Object string.</returns>
+        private string ToString(IDictionary<int, int> handled)
+        {
+            int idHash;
+
+            bool alreadyHandled = handled.TryGetValue(_offset, out idHash);
+
+            if (!alreadyHandled)
+                idHash = RuntimeHelpers.GetHashCode(this);
+
+            StringBuilder sb;
+
+            IPortableTypeDescriptor desc = _marsh.Descriptor(true, _typeId);
+
+            IPortableMetadata meta;
+
+            try
+            {
+                meta = _marsh.Metadata(_typeId);
+            }
+            catch (IgniteException)
+            {
+                meta = null;
+            }
+
+            if (meta == null)
+                sb = new StringBuilder("PortableObject [typeId=").Append(_typeId).Append(", idHash=" + idHash);
+            else
+            {
+                sb = new StringBuilder(meta.TypeName).Append(" [idHash=" + idHash);
+
+                if (!alreadyHandled)
+                {
+                    handled[_offset] = idHash;
+
+                    InitializeFields();
+                    
+                    foreach (string fieldName in meta.Fields)
+                    {
+                        sb.Append(", ");
+
+                        int fieldId = PortableUtils.FieldId(_typeId, fieldName, desc.NameConverter, desc.Mapper);
+
+                        int fieldPos;
+
+                        if (_fields.TryGetValue(fieldId, out fieldPos))
+                        {
+                            sb.Append(fieldName).Append('=');
+
+                            ToString0(sb, Field0<object>(fieldPos, null), handled);
+                        }
+                    }
+                }
+                else
+                    sb.Append(", ...");
+            }
+
+            sb.Append(']');
+
+            return sb.ToString();
+        }
+
+        /// <summary>
+        /// Internal ToString routine with correct collections printout.
+        /// </summary>
+        /// <param name="sb">String builder.</param>
+        /// <param name="obj">Object to print.</param>
+        /// <param name="handled">Already handled objects.</param>
+        /// <returns>The same string builder.</returns>
+        private static void ToString0(StringBuilder sb, object obj, IDictionary<int, int> handled)
+        {
+            IEnumerable col = (obj is string) ? null : obj as IEnumerable;
+
+            if (col == null)
+            {
+                PortableUserObject obj0 = obj as PortableUserObject;
+
+                sb.Append(obj0 == null ? obj : obj0.ToString(handled));
+            }
+            else
+            {
+                sb.Append('[');
+
+                bool first = true;
+
+                foreach (object elem in col)
+                {
+                    if (first)
+                        first = false;
+                    else
+                        sb.Append(", ");
+
+                    ToString0(sb, elem, handled);
+                }
+
+                sb.Append(']');
+            }
+        }
+    }
+}


[32/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEnumerator.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEnumerator.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEnumerator.cs
new file mode 100644
index 0000000..fd26558
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEnumerator.cs
@@ -0,0 +1,117 @@
+/*
+ * 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.Cache
+{
+    using System;
+    using System.Collections;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+
+    /// <summary>
+    /// Real cache enumerator communicating with Java.
+    /// </summary>
+    internal class CacheEnumerator<TK, TV> : PlatformDisposableTarget, IEnumerator<ICacheEntry<TK, TV>>
+    {
+        /** Operation: next value. */
+        private const int OpNext = 1;
+
+        /** Keep portable flag. */
+        private readonly bool _keepPortable;
+
+        /** Current entry. */
+        private CacheEntry<TK, TV>? _cur;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="target">Target.</param>
+        /// <param name="marsh">Marshaller.</param>
+        /// <param name="keepPortable">Keep portable flag.</param>
+        public CacheEnumerator(IUnmanagedTarget target, PortableMarshaller marsh, bool keepPortable) : 
+            base(target, marsh)
+        {
+            _keepPortable = keepPortable;
+        }
+
+        /** <inheritdoc /> */
+        public bool MoveNext()
+        {
+            ThrowIfDisposed();
+
+            return DoInOp(OpNext, stream =>
+            {
+                var reader = Marshaller.StartUnmarshal(stream, _keepPortable);
+
+                bool hasNext = reader.ReadBoolean();
+
+                if (hasNext)
+                {
+                    reader.DetachNext();
+                    TK key = reader.ReadObject<TK>();
+
+                    reader.DetachNext();
+                    TV val = reader.ReadObject<TV>();
+
+                    _cur = new CacheEntry<TK, TV>(key, val);
+
+                    return true;
+                }
+
+                _cur = null;
+
+                return false;
+            });
+        }
+
+        /** <inheritdoc /> */
+        public ICacheEntry<TK, TV> Current
+        {
+            get
+            {
+                ThrowIfDisposed();
+
+                if (_cur == null)
+                    throw new InvalidOperationException(
+                        "Invalid enumerator state, enumeration is either finished or not started");
+
+                return _cur.Value;
+            }
+        }
+
+        /** <inheritdoc /> */
+        object IEnumerator.Current
+        {
+            get { return Current; }
+        }
+
+        /** <inheritdoc /> */
+        public void Reset()
+        {
+            throw new NotSupportedException("Specified method is not supported.");
+        }
+
+        /** <inheritdoc /> */
+        protected override T Unmarshal<T>(IPortableStream stream)
+        {
+            throw new InvalidOperationException("Should not be called.");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEnumeratorProxy.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEnumeratorProxy.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEnumeratorProxy.cs
new file mode 100644
index 0000000..cadc58d
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEnumeratorProxy.cs
@@ -0,0 +1,156 @@
+/*
+ * 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.Cache
+{
+    using System;
+    using System.Collections;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Cache;
+
+    /// <summary>
+    /// Cache enumerator proxy. Required to support reset and early native iterator cleanup.
+    /// </summary>
+    internal class CacheEnumeratorProxy<TK, TV> : IEnumerator<ICacheEntry<TK, TV>>
+    {
+        /** Target cache. */
+        private readonly CacheImpl<TK, TV> _cache;
+
+        /** Local flag. */
+        private readonly bool _loc;
+
+        /** Peek modes. */
+        private readonly int _peekModes;
+
+        /** Target enumerator. */
+        private CacheEnumerator<TK, TV> _target;
+
+        /** Dispose flag. */
+        private bool _disposed;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="cache">Target cache.</param>
+        /// <param name="loc">Local flag.</param>
+        /// <param name="peekModes">Peek modes.</param>
+        public CacheEnumeratorProxy(CacheImpl<TK, TV> cache, bool loc, int peekModes)
+        {
+            _cache = cache;
+            _loc = loc;
+            _peekModes = peekModes;
+
+            CreateTarget();
+        }
+
+        /** <inheritdoc /> */
+        public bool MoveNext()
+        {
+            CheckDisposed();
+
+            // No target => closed or finished.
+            if (_target == null)
+                return false;
+            
+            if (!_target.MoveNext())
+            {
+                // Failed to advance => end is reached.
+                CloseTarget();
+
+                return false;
+            }
+
+            return true;
+        }
+
+        /** <inheritdoc /> */
+        public ICacheEntry<TK, TV> Current
+        {
+            get
+            {
+                CheckDisposed();
+
+                if (_target == null)
+                    throw new InvalidOperationException("Invalid enumerator state (did you call MoveNext()?)");
+
+                return _target.Current;
+            }
+        }
+
+        /** <inheritdoc /> */
+        object IEnumerator.Current
+        {
+            get { return Current; }
+        }
+
+        /** <inheritdoc /> */
+        public void Reset()
+        {
+            CheckDisposed();
+
+            if (_target != null)
+                CloseTarget();
+
+            CreateTarget();
+        }
+
+        /** <inheritdoc /> */
+        public void Dispose()
+        {
+            if (!_disposed)
+            {
+                if (_target != null)
+                    CloseTarget();
+
+                _disposed = true;
+            }
+        }
+
+        /// <summary>
+        /// Get target enumerator.
+        /// </summary>
+        /// <returns>Target enumerator.</returns>
+        private void CreateTarget()
+        {
+            Debug.Assert(_target == null, "Previous target is not cleaned.");
+
+            _target = _cache.CreateEnumerator(_loc, _peekModes);
+        }
+
+        /// <summary>
+        /// Close the target.
+        /// </summary>
+        private void CloseTarget()
+        {
+            Debug.Assert(_target != null);
+
+            _target.Dispose();
+
+            _target = null;
+        }
+
+        /// <summary>
+        /// Check whether object is disposed.
+        /// </summary>
+        private void CheckDisposed()
+        {
+            if (_disposed)
+                throw new ObjectDisposedException("Cache enumerator has been disposed.");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs
new file mode 100644
index 0000000..0301352
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs
@@ -0,0 +1,932 @@
+/*
+ * 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.Cache
+{
+    using System;
+    using System.Collections;
+    using System.Collections.Generic;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Threading;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cache.Expiry;
+    using Apache.Ignite.Core.Cache.Query;
+    using Apache.Ignite.Core.Cache.Query.Continuous;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Cache.Query;
+    using Apache.Ignite.Core.Impl.Cache.Query.Continuous;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+    using Apache.Ignite.Core.Portable;
+    using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils;
+
+    /// <summary>
+    /// Native cache wrapper.
+    /// </summary>
+    [SuppressMessage("Microsoft.Design", "CA1001:TypesThatOwnDisposableFieldsShouldBeDisposable")]
+    internal class CacheImpl<TK, TV> : PlatformTarget, ICache<TK, TV>
+    {
+        /** Duration: unchanged. */
+        private const long DurUnchanged = -2;
+
+        /** Duration: eternal. */
+        private const long DurEternal = -1;
+
+        /** Duration: zero. */
+        private const long DurZero = 0;
+
+        /** Ignite instance. */
+        private readonly Ignite _ignite;
+        
+        /** Flag: skip store. */
+        private readonly bool _flagSkipStore;
+
+        /** Flag: keep portable. */
+        private readonly bool _flagKeepPortable;
+
+        /** Flag: async mode.*/
+        private readonly bool _flagAsync;
+
+        /** Flag: no-retries.*/
+        private readonly bool _flagNoRetries;
+
+        /** 
+         * Result converter for async InvokeAll operation. 
+         * In future result processing there is only one TResult generic argument, 
+         * and we can't get the type of ICacheEntryProcessorResult at compile time from it.
+         * This field caches converter for the last InvokeAll operation to avoid using reflection.
+         */
+        private readonly ThreadLocal<object> _invokeAllConverter = new ThreadLocal<object>();
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="grid">Grid.</param>
+        /// <param name="target">Target.</param>
+        /// <param name="marsh">Marshaller.</param>
+        /// <param name="flagSkipStore">Skip store flag.</param>
+        /// <param name="flagKeepPortable">Keep portable flag.</param>
+        /// <param name="flagAsync">Async mode flag.</param>
+        /// <param name="flagNoRetries">No-retries mode flag.</param>
+        public CacheImpl(Ignite grid, IUnmanagedTarget target, PortableMarshaller marsh,
+            bool flagSkipStore, bool flagKeepPortable, bool flagAsync, bool flagNoRetries) : base(target, marsh)
+        {
+            _ignite = grid;
+            _flagSkipStore = flagSkipStore;
+            _flagKeepPortable = flagKeepPortable;
+            _flagAsync = flagAsync;
+            _flagNoRetries = flagNoRetries;
+        }
+
+        /** <inheritDoc /> */
+        public IIgnite Ignite
+        {
+            get
+            {
+                return _ignite;
+            }
+        }
+
+        /** <inheritDoc /> */
+        public bool IsAsync
+        {
+            get { return _flagAsync; }
+        }
+
+        /** <inheritDoc /> */
+        public IFuture GetFuture()
+        {
+            throw new NotSupportedException("GetFuture() should be called through CacheProxyImpl");
+        }
+
+        /** <inheritDoc /> */
+        public IFuture<TResult> GetFuture<TResult>()
+        {
+            throw new NotSupportedException("GetFuture() should be called through CacheProxyImpl");
+        }
+
+        /// <summary>
+        /// Gets and resets future for previous asynchronous operation.
+        /// </summary>
+        /// <param name="lastAsyncOpId">The last async op id.</param>
+        /// <returns>
+        /// Future for previous asynchronous operation.
+        /// </returns>
+        /// <exception cref="System.InvalidOperationException">Asynchronous mode is disabled</exception>
+        internal IFuture<TResult> GetFuture<TResult>(int lastAsyncOpId)
+        {
+            if (!_flagAsync)
+                throw IgniteUtils.GetAsyncModeDisabledException();
+
+            var converter = GetFutureResultConverter<TResult>(lastAsyncOpId);
+
+            _invokeAllConverter.Value = null;
+
+            return GetFuture((futId, futTypeId) => UU.TargetListenFutureForOperation(Target, futId, futTypeId, lastAsyncOpId), 
+                _flagKeepPortable, converter);
+        }
+
+        /** <inheritDoc /> */
+        public string Name
+        {
+            get { return DoInOp<string>((int)CacheOp.GetName); }
+        }
+
+        /** <inheritDoc /> */
+        public bool IsEmpty
+        {
+            get { return Size() == 0; }
+        }
+
+        /** <inheritDoc /> */
+        public ICache<TK, TV> WithSkipStore()
+        {
+            if (_flagSkipStore)
+                return this;
+
+            return new CacheImpl<TK, TV>(_ignite, UU.CacheWithSkipStore(Target), Marshaller, 
+                true, _flagKeepPortable, _flagAsync, true);
+        }
+
+        /// <summary>
+        /// Skip store flag getter.
+        /// </summary>
+        internal bool IsSkipStore { get { return _flagSkipStore; } }
+
+        /** <inheritDoc /> */
+        public ICache<TK1, TV1> WithKeepPortable<TK1, TV1>()
+        {
+            if (_flagKeepPortable)
+            {
+                var result = this as ICache<TK1, TV1>;
+
+                if (result == null)
+                    throw new InvalidOperationException(
+                        "Can't change type of portable cache. WithKeepPortable has been called on an instance of " +
+                        "portable cache with incompatible generic arguments.");
+
+                return result;
+            }
+
+            return new CacheImpl<TK1, TV1>(_ignite, UU.CacheWithKeepPortable(Target), Marshaller, 
+                _flagSkipStore, true, _flagAsync, _flagNoRetries);
+        }
+
+        /** <inheritDoc /> */
+        public ICache<TK, TV> WithExpiryPolicy(IExpiryPolicy plc)
+        {
+            IgniteArgumentCheck.NotNull(plc, "plc");
+
+            long create = ConvertDuration(plc.GetExpiryForCreate());
+            long update = ConvertDuration(plc.GetExpiryForUpdate());
+            long access = ConvertDuration(plc.GetExpiryForAccess());
+
+            IUnmanagedTarget cache0 = UU.CacheWithExpiryPolicy(Target, create, update, access);
+
+            return new CacheImpl<TK, TV>(_ignite, cache0, Marshaller, _flagSkipStore, _flagKeepPortable, _flagAsync, _flagNoRetries);
+        }
+
+        /// <summary>
+        /// Convert TimeSpan to duration recognizable by Java.
+        /// </summary>
+        /// <param name="dur">.Net duration.</param>
+        /// <returns>Java duration in milliseconds.</returns>
+        private static long ConvertDuration(TimeSpan? dur)
+        {
+            if (dur.HasValue)
+            {
+                if (dur.Value == TimeSpan.MaxValue)
+                    return DurEternal;
+
+                long dur0 = (long)dur.Value.TotalMilliseconds;
+
+                return dur0 > 0 ? dur0 : DurZero;
+            }
+            
+            return DurUnchanged;
+        }
+
+        /** <inheritDoc /> */
+        public ICache<TK, TV> WithAsync()
+        {
+            return _flagAsync ? this : new CacheImpl<TK, TV>(_ignite, UU.CacheWithAsync(Target), Marshaller,
+                _flagSkipStore, _flagKeepPortable, true, _flagNoRetries);
+        }
+
+        /** <inheritDoc /> */
+        public bool KeepPortable
+        {
+            get { return _flagKeepPortable; }
+        }
+
+        /** <inheritDoc /> */
+        public void LoadCache(ICacheEntryFilter<TK, TV> p, params object[] args)
+        {
+            LoadCache0(p, args, (int)CacheOp.LoadCache);
+        }
+
+        /** <inheritDoc /> */
+        public void LocalLoadCache(ICacheEntryFilter<TK, TV> p, params object[] args)
+        {
+            LoadCache0(p, args, (int)CacheOp.LocLoadCache);
+        }
+
+        /// <summary>
+        /// Loads the cache.
+        /// </summary>
+        private void LoadCache0(ICacheEntryFilter<TK, TV> p, object[] args, int opId)
+        {
+            DoOutOp(opId, writer =>
+            {
+                if (p != null)
+                {
+                    var p0 = new CacheEntryFilterHolder(p, (k, v) => p.Invoke(new CacheEntry<TK, TV>((TK)k, (TV)v)),
+                        Marshaller, KeepPortable);
+                    writer.WriteObject(p0);
+                    writer.WriteLong(p0.Handle);
+                }
+                else
+                    writer.WriteObject<CacheEntryFilterHolder>(null);
+
+                writer.WriteObjectArray(args);
+            });
+        }
+
+        /** <inheritDoc /> */
+        public bool ContainsKey(TK key)
+        {
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            return DoOutOp((int)CacheOp.ContainsKey, key) == True;
+        }        
+
+        /** <inheritDoc /> */
+        public bool ContainsKeys(IEnumerable<TK> keys)
+        {
+            IgniteArgumentCheck.NotNull(keys, "keys");
+
+            return DoOutOp((int)CacheOp.ContainsKeys, writer => WriteEnumerable(writer, keys)) == True;
+        }        
+
+        /** <inheritDoc /> */
+        public TV LocalPeek(TK key, params CachePeekMode[] modes)
+        {
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            return DoOutInOp<TV>((int)CacheOp.Peek, writer =>
+            {
+                writer.Write(key);
+                writer.WriteInt(EncodePeekModes(modes));
+            });
+        }
+
+        /** <inheritDoc /> */
+        public TV Get(TK key)
+        {
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            return DoOutInOp<TK, TV>((int)CacheOp.Get, key);
+        }
+
+        /** <inheritDoc /> */
+        public IDictionary<TK, TV> GetAll(IEnumerable<TK> keys)
+        {
+            IgniteArgumentCheck.NotNull(keys, "keys");
+
+            return DoOutInOp((int)CacheOp.GetAll,
+                writer => WriteEnumerable(writer, keys),
+                input =>
+                {
+                    var reader = Marshaller.StartUnmarshal(input, _flagKeepPortable);
+
+                    return ReadGetAllDictionary(reader);
+                });
+        }
+
+        /** <inheritdoc /> */
+        public void Put(TK key, TV val)
+        {
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            IgniteArgumentCheck.NotNull(val, "val");
+
+            DoOutOp((int)CacheOp.Put, key, val);
+        }
+
+        /** <inheritDoc /> */
+        public TV GetAndPut(TK key, TV val)
+        {
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            IgniteArgumentCheck.NotNull(val, "val");
+
+            return DoOutInOp<TK, TV, TV>((int)CacheOp.GetAndPut, key, val);
+        }
+
+        /** <inheritDoc /> */
+        public TV GetAndReplace(TK key, TV val)
+        {
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            IgniteArgumentCheck.NotNull(val, "val");
+
+            return DoOutInOp<TK, TV, TV>((int)CacheOp.GetAndReplace, key, val);
+        }
+
+        /** <inheritDoc /> */
+        public TV GetAndRemove(TK key)
+        {
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            return DoOutInOp<TK, TV>((int)CacheOp.GetAndRemove, key);
+        }
+
+        /** <inheritdoc /> */
+        public bool PutIfAbsent(TK key, TV val)
+        {
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            IgniteArgumentCheck.NotNull(val, "val");
+
+            return DoOutOp((int) CacheOp.PutIfAbsent, key, val) == True;
+        }
+
+        /** <inheritdoc /> */
+        public TV GetAndPutIfAbsent(TK key, TV val)
+        {
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            IgniteArgumentCheck.NotNull(val, "val");
+
+            return DoOutInOp<TK, TV, TV>((int)CacheOp.GetAndPutIfAbsent, key, val);
+        }
+
+        /** <inheritdoc /> */
+        public bool Replace(TK key, TV val)
+        {
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            IgniteArgumentCheck.NotNull(val, "val");
+
+            return DoOutOp((int)CacheOp.Replace2, key, val) == True;
+        }
+
+        /** <inheritdoc /> */
+        public bool Replace(TK key, TV oldVal, TV newVal)
+        {
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            IgniteArgumentCheck.NotNull(oldVal, "oldVal");
+
+            IgniteArgumentCheck.NotNull(newVal, "newVal");
+
+            return DoOutOp((int)CacheOp.Replace3, key, oldVal, newVal) == True;
+        }
+
+        /** <inheritdoc /> */
+        public void PutAll(IDictionary<TK, TV> vals)
+        {
+            IgniteArgumentCheck.NotNull(vals, "vals");
+
+            DoOutOp((int) CacheOp.PutAll, writer => WriteDictionary(writer, vals));
+        }
+        
+        /** <inheritdoc /> */
+        public void LocalEvict(IEnumerable<TK> keys)
+        {
+            IgniteArgumentCheck.NotNull(keys, "keys");
+
+            DoOutOp((int) CacheOp.LocEvict, writer => WriteEnumerable(writer, keys));
+        }
+
+        /** <inheritdoc /> */
+        public void Clear()
+        {
+            UU.CacheClear(Target);
+        }
+
+        /** <inheritdoc /> */
+        public void Clear(TK key)
+        {
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            DoOutOp((int)CacheOp.Clear, key);
+        }
+
+        /** <inheritdoc /> */
+        public void ClearAll(IEnumerable<TK> keys)
+        {
+            IgniteArgumentCheck.NotNull(keys, "keys");
+
+            DoOutOp((int)CacheOp.ClearAll, writer => WriteEnumerable(writer, keys));
+        }
+
+        /** <inheritdoc /> */
+        public void LocalClear(TK key)
+        {
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            DoOutOp((int) CacheOp.LocalClear, key);
+        }
+
+        /** <inheritdoc /> */
+        public void LocalClearAll(IEnumerable<TK> keys)
+        {
+            IgniteArgumentCheck.NotNull(keys, "keys");
+
+            DoOutOp((int)CacheOp.LocalClearAll, writer => WriteEnumerable(writer, keys));
+        }
+
+        /** <inheritdoc /> */
+        public bool Remove(TK key)
+        {
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            return DoOutOp((int)CacheOp.RemoveObj, key) == True;
+        }
+
+        /** <inheritDoc /> */
+        public bool Remove(TK key, TV val)
+        {
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            IgniteArgumentCheck.NotNull(val, "val");
+
+            return DoOutOp((int)CacheOp.RemoveBool, key, val) == True;
+        }
+
+        /** <inheritDoc /> */
+        public void RemoveAll(IEnumerable<TK> keys)
+        {
+            IgniteArgumentCheck.NotNull(keys, "keys");
+
+            DoOutOp((int)CacheOp.RemoveAll, writer => WriteEnumerable(writer, keys));
+        }
+
+        /** <inheritDoc /> */
+        public void RemoveAll()
+        {
+            UU.CacheRemoveAll(Target);
+        }
+
+        /** <inheritDoc /> */
+        public int LocalSize(params CachePeekMode[] modes)
+        {
+            return Size0(true, modes);
+        }
+
+        /** <inheritDoc /> */
+        public int Size(params CachePeekMode[] modes)
+        {
+            return Size0(false, modes);
+        }
+
+        /// <summary>
+        /// Internal size routine.
+        /// </summary>
+        /// <param name="loc">Local flag.</param>
+        /// <param name="modes">peek modes</param>
+        /// <returns>Size.</returns>
+        private int Size0(bool loc, params CachePeekMode[] modes)
+        {
+            int modes0 = EncodePeekModes(modes);
+
+            return UU.CacheSize(Target, modes0, loc);
+        }
+
+        /** <inheritDoc /> */
+        public void LocalPromote(IEnumerable<TK> keys)
+        {
+            IgniteArgumentCheck.NotNull(keys, "keys");
+
+            DoOutOp((int)CacheOp.LocPromote, writer => WriteEnumerable(writer, keys));
+        }
+
+        /** <inheritdoc /> */
+        public TR Invoke<TR, TA>(TK key, ICacheEntryProcessor<TK, TV, TA, TR> processor, TA arg)
+        {
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            IgniteArgumentCheck.NotNull(processor, "processor");
+
+            var holder = new CacheEntryProcessorHolder(processor, arg,
+                (e, a) => processor.Process((IMutableCacheEntry<TK, TV>)e, (TA)a), typeof(TK), typeof(TV));
+
+            return DoOutInOp((int)CacheOp.Invoke, writer =>
+            {
+                writer.Write(key);
+                writer.Write(holder);
+            },
+            input => GetResultOrThrow<TR>(Unmarshal<object>(input)));
+        }
+
+        /** <inheritdoc /> */
+        public IDictionary<TK, ICacheEntryProcessorResult<TR>> InvokeAll<TR, TA>(IEnumerable<TK> keys,
+            ICacheEntryProcessor<TK, TV, TA, TR> processor, TA arg)
+        {
+            IgniteArgumentCheck.NotNull(keys, "keys");
+
+            IgniteArgumentCheck.NotNull(processor, "processor");
+
+            var holder = new CacheEntryProcessorHolder(processor, arg,
+                (e, a) => processor.Process((IMutableCacheEntry<TK, TV>)e, (TA)a), typeof(TK), typeof(TV));
+
+            return DoOutInOp((int)CacheOp.InvokeAll, writer =>
+            {
+                WriteEnumerable(writer, keys);
+                writer.Write(holder);
+            },
+            input =>
+            {
+                if (IsAsync)
+                    _invokeAllConverter.Value = (Func<PortableReaderImpl, IDictionary<TK, ICacheEntryProcessorResult<TR>>>)
+                        (reader => ReadInvokeAllResults<TR>(reader.Stream));
+
+                return ReadInvokeAllResults<TR>(input);
+            });
+        }
+
+        /** <inheritdoc /> */
+        public ICacheLock Lock(TK key)
+        {
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            return DoOutInOp((int)CacheOp.Lock, writer =>
+            {
+                writer.Write(key);
+            }, input => new CacheLock(input.ReadInt(), Target));
+        }
+
+        /** <inheritdoc /> */
+        public ICacheLock LockAll(IEnumerable<TK> keys)
+        {
+            IgniteArgumentCheck.NotNull(keys, "keys");
+
+            return DoOutInOp((int)CacheOp.LockAll, writer =>
+            {
+                WriteEnumerable(writer, keys);
+            }, input => new CacheLock(input.ReadInt(), Target));
+        }
+
+        /** <inheritdoc /> */
+        public bool IsLocalLocked(TK key, bool byCurrentThread)
+        {
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            return DoOutOp((int)CacheOp.IsLocalLocked, writer =>
+            {
+                writer.Write(key);
+                writer.WriteBoolean(byCurrentThread);
+            }) == True;
+        }
+
+        /** <inheritDoc /> */
+        public ICacheMetrics GetMetrics()
+        {
+            return DoInOp((int)CacheOp.Metrics, stream =>
+            {
+                IPortableRawReader reader = Marshaller.StartUnmarshal(stream, false);
+
+                return new CacheMetricsImpl(reader);
+            });
+        }
+
+        /** <inheritDoc /> */
+        public IFuture Rebalance()
+        {
+            return GetFuture<object>((futId, futTyp) => UU.CacheRebalance(Target, futId));
+        }
+
+        /** <inheritDoc /> */
+        public ICache<TK, TV> WithNoRetries()
+        {
+            if (_flagNoRetries)
+                return this;
+
+            return new CacheImpl<TK, TV>(_ignite, UU.CacheWithNoRetries(Target), Marshaller,
+                _flagSkipStore, _flagKeepPortable, _flagAsync, true);
+        }
+
+        /// <summary>
+        /// Gets a value indicating whether this instance is in no-retries mode.
+        /// </summary>
+        internal bool IsNoRetries
+        {
+            get { return _flagNoRetries; }
+        }
+
+        #region Queries
+
+        /** <inheritDoc /> */
+        public IQueryCursor<IList> QueryFields(SqlFieldsQuery qry)
+        {
+            IgniteArgumentCheck.NotNull(qry, "qry");
+
+            if (string.IsNullOrEmpty(qry.Sql))
+                throw new ArgumentException("Sql cannot be null or empty");
+
+            IUnmanagedTarget cursor;
+
+            using (var stream = IgniteManager.Memory.Allocate().Stream())
+            {
+                var writer = Marshaller.StartMarshal(stream);
+
+                writer.WriteBoolean(qry.Local);
+                writer.WriteString(qry.Sql);
+                writer.WriteInt(qry.PageSize);
+
+                WriteQueryArgs(writer, qry.Arguments);
+
+                FinishMarshal(writer);
+
+                cursor = UU.CacheOutOpQueryCursor(Target, (int) CacheOp.QrySqlFields, stream.SynchronizeOutput());
+            }
+        
+            return new FieldsQueryCursor(cursor, Marshaller, _flagKeepPortable);
+        }
+
+        /** <inheritDoc /> */
+        public IQueryCursor<ICacheEntry<TK, TV>> Query(QueryBase qry)
+        {
+            IgniteArgumentCheck.NotNull(qry, "qry");
+
+            IUnmanagedTarget cursor;
+
+            using (var stream = IgniteManager.Memory.Allocate().Stream())
+            {
+                var writer = Marshaller.StartMarshal(stream);
+
+                qry.Write(writer, KeepPortable);
+
+                FinishMarshal(writer);
+
+                cursor = UU.CacheOutOpQueryCursor(Target, (int)qry.OpId, stream.SynchronizeOutput()); 
+            }
+
+            return new QueryCursor<TK, TV>(cursor, Marshaller, _flagKeepPortable);
+        }
+                
+        /// <summary>
+        /// Write query arguments.
+        /// </summary>
+        /// <param name="writer">Writer.</param>
+        /// <param name="args">Arguments.</param>
+        private static void WriteQueryArgs(PortableWriterImpl writer, object[] args)
+        {
+            if (args == null)
+                writer.WriteInt(0);
+            else
+            {
+                writer.WriteInt(args.Length);
+        
+                foreach (var arg in args)
+                    writer.WriteObject(arg);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public IContinuousQueryHandle QueryContinuous(ContinuousQuery<TK, TV> qry)
+        {
+            IgniteArgumentCheck.NotNull(qry, "qry");
+
+            return QueryContinuousImpl(qry, null);
+        }
+
+        /** <inheritdoc /> */
+        public IContinuousQueryHandle<ICacheEntry<TK, TV>> QueryContinuous(ContinuousQuery<TK, TV> qry, QueryBase initialQry)
+        {
+            IgniteArgumentCheck.NotNull(qry, "qry");
+            IgniteArgumentCheck.NotNull(initialQry, "initialQry");
+
+            return QueryContinuousImpl(qry, initialQry);
+        }
+
+        /// <summary>
+        /// QueryContinuous implementation.
+        /// </summary>
+        private IContinuousQueryHandle<ICacheEntry<TK, TV>> QueryContinuousImpl(ContinuousQuery<TK, TV> qry, 
+            QueryBase initialQry)
+        {
+            qry.Validate();
+
+            var hnd = new ContinuousQueryHandleImpl<TK, TV>(qry, Marshaller, _flagKeepPortable);
+
+            using (var stream = IgniteManager.Memory.Allocate().Stream())
+            {
+                var writer = Marshaller.StartMarshal(stream);
+
+                hnd.Start(_ignite, writer, () =>
+                {
+                    if (initialQry != null)
+                    {
+                        writer.WriteInt((int) initialQry.OpId);
+
+                        initialQry.Write(writer, KeepPortable);
+                    }
+                    else
+                        writer.WriteInt(-1); // no initial query
+
+                    FinishMarshal(writer);
+
+                    // ReSharper disable once AccessToDisposedClosure
+                    return UU.CacheOutOpContinuousQuery(Target, (int)CacheOp.QryContinuous, stream.SynchronizeOutput());
+                }, qry);
+            }
+
+            return hnd;
+        }
+
+        #endregion
+
+        #region Enumerable support
+
+        /** <inheritdoc /> */
+        public IEnumerable<ICacheEntry<TK, TV>> GetLocalEntries(CachePeekMode[] peekModes)
+        {
+            return new CacheEnumerable<TK, TV>(this, EncodePeekModes(peekModes));
+        }
+
+        /** <inheritdoc /> */
+        public IEnumerator<ICacheEntry<TK, TV>> GetEnumerator()
+        {
+            return new CacheEnumeratorProxy<TK, TV>(this, false, 0);
+        }
+
+        /** <inheritdoc /> */
+        IEnumerator IEnumerable.GetEnumerator()
+        {
+            return GetEnumerator();
+        }
+
+        /// <summary>
+        /// Create real cache enumerator.
+        /// </summary>
+        /// <param name="loc">Local flag.</param>
+        /// <param name="peekModes">Peek modes for local enumerator.</param>
+        /// <returns>Cache enumerator.</returns>
+        internal CacheEnumerator<TK, TV> CreateEnumerator(bool loc, int peekModes)
+        {
+            if (loc)
+                return new CacheEnumerator<TK, TV>(UU.CacheLocalIterator(Target, peekModes), Marshaller, _flagKeepPortable);
+
+            return new CacheEnumerator<TK, TV>(UU.CacheIterator(Target), Marshaller, _flagKeepPortable);
+        }
+
+        #endregion
+
+        /** <inheritDoc /> */
+        protected override T Unmarshal<T>(IPortableStream stream)
+        {
+            return Marshaller.Unmarshal<T>(stream, _flagKeepPortable);
+        }
+
+        /// <summary>
+        /// Encodes the peek modes into a single int value.
+        /// </summary>
+        private static int EncodePeekModes(CachePeekMode[] modes)
+        {
+            int modesEncoded = 0;
+
+            if (modes != null)
+            {
+                foreach (var mode in modes)
+                    modesEncoded |= (int) mode;
+            }
+
+            return modesEncoded;
+        }
+
+        /// <summary>
+        /// Unwraps an exception from PortableResultHolder, if any. Otherwise does the cast.
+        /// </summary>
+        /// <typeparam name="T">Result type.</typeparam>
+        /// <param name="obj">Object.</param>
+        /// <returns>Result.</returns>
+        private static T GetResultOrThrow<T>(object obj)
+        {
+            var holder = obj as PortableResultWrapper;
+
+            if (holder != null)
+            {
+                var err = holder.Result as Exception;
+
+                if (err != null)
+                    throw err as CacheEntryProcessorException ?? new CacheEntryProcessorException(err);
+            }
+
+            return obj == null ? default(T) : (T) obj;
+        }
+
+        /// <summary>
+        /// Reads results of InvokeAll operation.
+        /// </summary>
+        /// <typeparam name="T">The type of the result.</typeparam>
+        /// <param name="inStream">Stream.</param>
+        /// <returns>Results of InvokeAll operation.</returns>
+        private IDictionary<TK, ICacheEntryProcessorResult<T>> ReadInvokeAllResults<T>(IPortableStream inStream)
+        {
+            var count = inStream.ReadInt();
+
+            if (count == -1)
+                return null;
+
+            var results = new Dictionary<TK, ICacheEntryProcessorResult<T>>(count);
+
+            for (var i = 0; i < count; i++)
+            {
+                var key = Unmarshal<TK>(inStream);
+
+                var hasError = inStream.ReadBool();
+
+                results[key] = hasError
+                    ? new CacheEntryProcessorResult<T>(ReadException(inStream))
+                    : new CacheEntryProcessorResult<T>(Unmarshal<T>(inStream));
+            }
+
+            return results;
+        }
+
+        /// <summary>
+        /// Reads the exception, either in portable wrapper form, or as a pair of strings.
+        /// </summary>
+        /// <param name="inStream">The stream.</param>
+        /// <returns>Exception.</returns>
+        private CacheEntryProcessorException ReadException(IPortableStream inStream)
+        {
+            var item = Unmarshal<object>(inStream);
+
+            var clsName = item as string;
+
+            if (clsName == null)
+                return new CacheEntryProcessorException((Exception) ((PortableResultWrapper) item).Result);
+
+            var msg = Unmarshal<string>(inStream);
+                
+            return new CacheEntryProcessorException(ExceptionUtils.GetException(clsName, msg));
+        }
+
+        /// <summary>
+        /// Read dictionary returned by GET_ALL operation.
+        /// </summary>
+        /// <param name="reader">Reader.</param>
+        /// <returns>Dictionary.</returns>
+        private static IDictionary<TK, TV> ReadGetAllDictionary(PortableReaderImpl reader)
+        {
+            IPortableStream stream = reader.Stream;
+
+            if (stream.ReadBool())
+            {
+                int size = stream.ReadInt();
+
+                IDictionary<TK, TV> res = new Dictionary<TK, TV>(size);
+
+                for (int i = 0; i < size; i++)
+                {
+                    TK key = reader.ReadObject<TK>();
+                    TV val = reader.ReadObject<TV>();
+
+                    res[key] = val;
+                }
+
+                return res;
+            }
+            return null;
+        }
+
+        /// <summary>
+        /// Gets the future result converter based on the last operation id.
+        /// </summary>
+        /// <typeparam name="TResult">The type of the future result.</typeparam>
+        /// <param name="lastAsyncOpId">The last op id.</param>
+        /// <returns>Future result converter.</returns>
+        private Func<PortableReaderImpl, TResult> GetFutureResultConverter<TResult>(int lastAsyncOpId)
+        {
+            if (lastAsyncOpId == (int) CacheOp.GetAll)
+                return reader => (TResult)ReadGetAllDictionary(reader);
+            
+            if (lastAsyncOpId == (int)CacheOp.Invoke)
+                return reader => { throw ReadException(reader.Stream); };
+
+            if (lastAsyncOpId == (int) CacheOp.InvokeAll)
+                return _invokeAllConverter.Value as Func<PortableReaderImpl, TResult>;
+
+            return null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheLock.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheLock.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheLock.cs
new file mode 100644
index 0000000..ceb3b05
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheLock.cs
@@ -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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Cache
+{
+    using System;
+    using System.Diagnostics;
+    using System.Threading;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+    using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils;
+
+    /// <summary>
+    /// Cache lock implementation.
+    /// </summary>
+    internal class CacheLock : ICacheLock
+    {
+        /** Unique lock ID.*/
+        private readonly long _id;
+
+        /** Cache. */
+        private readonly IUnmanagedTarget _cache;
+
+        /** State (-1 for disposed, >=0 for number of currently executing methods). */
+        private int _state;
+
+        /** Current number of lock contenders. */
+        private int _counter;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheLock"/> class.
+        /// </summary>
+        /// <param name="id">Lock id.</param>
+        /// <param name="cache">Cache.</param>
+        public CacheLock(long id, IUnmanagedTarget cache)
+        {
+            Debug.Assert(cache != null);
+
+            _id = id;
+            _cache = cache;
+        }
+
+        /** <inheritDoc /> */
+        public void Enter()
+        {
+            lock (this)
+            {
+                ThrowIfDisposed();
+
+                _state++;
+            }
+
+            var res = false;
+
+            try
+            {
+                UU.CacheEnterLock(_cache, _id);
+
+                res = true;
+            }
+            finally 
+            {
+                lock (this)
+                {
+                    if (res)
+                        _counter++;
+
+                    _state--;
+                }
+            }
+        }
+
+        /** <inheritDoc /> */
+        public bool TryEnter()
+        {
+            return TryEnter(TimeSpan.FromMilliseconds(-1));
+        }
+
+        /** <inheritDoc /> */
+        public bool TryEnter(TimeSpan timeout)
+        {
+            lock (this)
+            {
+                ThrowIfDisposed();
+
+                _state++;
+            }
+            
+            var res = false;
+
+            try
+            {
+                return res = UU.CacheTryEnterLock(_cache, _id, (long)timeout.TotalMilliseconds);
+            }
+            finally 
+            {
+                lock (this)
+                {
+                    if (res)
+                        _counter++;
+
+                    _state--;
+                }
+            }
+        }
+
+        /** <inheritDoc /> */
+        public void Exit()
+        {
+            lock (this)
+            {
+                ThrowIfDisposed();
+
+                UU.CacheExitLock(_cache, _id);
+
+                _counter--;
+            }
+        }
+
+        /** <inheritDoc /> */
+        public void Dispose()
+        {
+            lock (this)
+            {
+                ThrowIfDisposed();
+
+                if (_state > 0 || _counter > 0)
+                    throw new SynchronizationLockException(
+                        "The lock is being disposed while still being used. " +
+                        "It either is being held by a thread and/or has active waiters waiting to acquire the lock.");
+
+                UU.CacheCloseLock(_cache, _id);
+
+                _state = -1;
+
+                GC.SuppressFinalize(this);
+            }
+        }
+
+        /// <summary>
+        /// Finalizes an instance of the <see cref="CacheLock"/> class.
+        /// </summary>
+        ~CacheLock()
+        {
+            UU.CacheCloseLock(_cache, _id);
+        }
+
+        /// <summary>
+        /// Throws <see cref="ObjectDisposedException"/> if this instance has been disposed.
+        /// </summary>
+        private void ThrowIfDisposed()
+        {
+            if (_state < 0)
+                throw new ObjectDisposedException("CacheLock", "CacheLock has been disposed.");
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs
new file mode 100644
index 0000000..b5982f6
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs
@@ -0,0 +1,248 @@
+/*
+ * 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.Cache
+{
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Cache metrics used to obtain statistics on cache.
+    /// </summary>
+    internal class CacheMetricsImpl : ICacheMetrics
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheMetricsImpl"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        public CacheMetricsImpl(IPortableRawReader reader)
+        {
+            CacheGets = reader.ReadLong();
+            CachePuts = reader.ReadLong();
+            CacheHits = reader.ReadLong();
+            CacheMisses = reader.ReadLong();
+            CacheTxCommits = reader.ReadLong();
+            CacheTxRollbacks = reader.ReadLong();
+            CacheEvictions = reader.ReadLong();
+            CacheRemovals = reader.ReadLong();
+            AveragePutTime = reader.ReadFloat();
+            AverageGetTime = reader.ReadFloat();
+            AverageRemoveTime = reader.ReadFloat();
+            AverageTxCommitTime = reader.ReadFloat();
+            AverageTxRollbackTime = reader.ReadFloat();
+            CacheName = reader.ReadString();
+            OverflowSize = reader.ReadLong();
+            OffHeapEntriesCount = reader.ReadLong();
+            OffHeapAllocatedSize = reader.ReadLong();
+            Size = reader.ReadInt();
+            KeySize = reader.ReadInt();
+            IsEmpty = reader.ReadBoolean();
+            DhtEvictQueueCurrentSize = reader.ReadInt();
+            TxThreadMapSize = reader.ReadInt();
+            TxXidMapSize = reader.ReadInt();
+            TxCommitQueueSize = reader.ReadInt();
+            TxPrepareQueueSize = reader.ReadInt();
+            TxStartVersionCountsSize = reader.ReadInt();
+            TxCommittedVersionsSize = reader.ReadInt();
+            TxRolledbackVersionsSize = reader.ReadInt();
+            TxDhtThreadMapSize = reader.ReadInt();
+            TxDhtXidMapSize = reader.ReadInt();
+            TxDhtCommitQueueSize = reader.ReadInt();
+            TxDhtPrepareQueueSize = reader.ReadInt();
+            TxDhtStartVersionCountsSize = reader.ReadInt();
+            TxDhtCommittedVersionsSize = reader.ReadInt();
+            TxDhtRolledbackVersionsSize = reader.ReadInt();
+            IsWriteBehindEnabled = reader.ReadBoolean();
+            WriteBehindFlushSize = reader.ReadInt();
+            WriteBehindFlushThreadCount = reader.ReadInt();
+            WriteBehindFlushFrequency = reader.ReadLong();
+            WriteBehindStoreBatchSize = reader.ReadInt();
+            WriteBehindTotalCriticalOverflowCount = reader.ReadInt();
+            WriteBehindCriticalOverflowCount = reader.ReadInt();
+            WriteBehindErrorRetryCount = reader.ReadInt();
+            WriteBehindBufferSize = reader.ReadInt();
+            KeyType = reader.ReadString();
+            ValueType = reader.ReadString();
+            IsStoreByValue = reader.ReadBoolean();
+            IsStatisticsEnabled = reader.ReadBoolean();
+            IsManagementEnabled = reader.ReadBoolean();
+            IsReadThrough = reader.ReadBoolean();
+            IsWriteThrough = reader.ReadBoolean();
+            CacheHitPercentage = reader.ReadFloat();
+            CacheMissPercentage = reader.ReadFloat();
+        }
+
+        /** <inheritdoc /> */
+        public long CacheHits { get; private set; }
+
+        /** <inheritdoc /> */
+        public float CacheHitPercentage { get; private set; }
+
+        /** <inheritdoc /> */
+        public long CacheMisses { get; private set; }
+
+        /** <inheritdoc /> */
+        public float CacheMissPercentage { get; private set; }
+
+        /** <inheritdoc /> */
+        public long CacheGets { get; private set; }
+
+        /** <inheritdoc /> */
+        public long CachePuts { get; private set; }
+
+        /** <inheritdoc /> */
+        public long CacheRemovals { get; private set; }
+
+        /** <inheritdoc /> */
+        public long CacheEvictions { get; private set; }
+
+        /** <inheritdoc /> */
+        public float AverageGetTime { get; private set; }
+
+        /** <inheritdoc /> */
+        public float AveragePutTime { get; private set; }
+
+        /** <inheritdoc /> */
+        public float AverageRemoveTime { get; private set; }
+
+        /** <inheritdoc /> */
+        public float AverageTxCommitTime { get; private set; }
+
+        /** <inheritdoc /> */
+        public float AverageTxRollbackTime { get; private set; }
+
+        /** <inheritdoc /> */
+        public long CacheTxCommits { get; private set; }
+
+        /** <inheritdoc /> */
+        public long CacheTxRollbacks { get; private set; }
+
+        /** <inheritdoc /> */
+        public string CacheName { get; private set; }
+
+        /** <inheritdoc /> */
+        public long OverflowSize { get; private set; }
+
+        /** <inheritdoc /> */
+        public long OffHeapEntriesCount { get; private set; }
+
+        /** <inheritdoc /> */
+        public long OffHeapAllocatedSize { get; private set; }
+
+        /** <inheritdoc /> */
+        public int Size { get; private set; }
+
+        /** <inheritdoc /> */
+        public int KeySize { get; private set; }
+
+        /** <inheritdoc /> */
+        public bool IsEmpty { get; private set; }
+
+        /** <inheritdoc /> */
+        public int DhtEvictQueueCurrentSize { get; private set; }
+
+        /** <inheritdoc /> */
+        public int TxThreadMapSize { get; private set; }
+
+        /** <inheritdoc /> */
+        public int TxXidMapSize { get; private set; }
+
+        /** <inheritdoc /> */
+        public int TxCommitQueueSize { get; private set; }
+
+        /** <inheritdoc /> */
+        public int TxPrepareQueueSize { get; private set; }
+
+        /** <inheritdoc /> */
+        public int TxStartVersionCountsSize { get; private set; }
+
+        /** <inheritdoc /> */
+        public int TxCommittedVersionsSize { get; private set; }
+
+        /** <inheritdoc /> */
+        public int TxRolledbackVersionsSize { get; private set; }
+
+        /** <inheritdoc /> */
+        public int TxDhtThreadMapSize { get; private set; }
+
+        /** <inheritdoc /> */
+        public int TxDhtXidMapSize { get; private set; }
+
+        /** <inheritdoc /> */
+        public int TxDhtCommitQueueSize { get; private set; }
+
+        /** <inheritdoc /> */
+        public int TxDhtPrepareQueueSize { get; private set; }
+
+        /** <inheritdoc /> */
+        public int TxDhtStartVersionCountsSize { get; private set; }
+
+        /** <inheritdoc /> */
+        public int TxDhtCommittedVersionsSize { get; private set; }
+
+        /** <inheritdoc /> */
+        public int TxDhtRolledbackVersionsSize { get; private set; }
+
+        /** <inheritdoc /> */
+        public bool IsWriteBehindEnabled { get; private set; }
+
+        /** <inheritdoc /> */
+        public int WriteBehindFlushSize { get; private set; }
+
+        /** <inheritdoc /> */
+        public int WriteBehindFlushThreadCount { get; private set; }
+
+        /** <inheritdoc /> */
+        public long WriteBehindFlushFrequency { get; private set; }
+
+        /** <inheritdoc /> */
+        public int WriteBehindStoreBatchSize { get; private set; }
+
+        /** <inheritdoc /> */
+        public int WriteBehindTotalCriticalOverflowCount { get; private set; }
+
+        /** <inheritdoc /> */
+        public int WriteBehindCriticalOverflowCount { get; private set; }
+
+        /** <inheritdoc /> */
+        public int WriteBehindErrorRetryCount { get; private set; }
+
+        /** <inheritdoc /> */
+        public int WriteBehindBufferSize { get; private set; }
+
+        /** <inheritdoc /> */
+        public string KeyType { get; private set; }
+
+        /** <inheritdoc /> */
+        public string ValueType { get; private set; }
+
+        /** <inheritdoc /> */
+        public bool IsStoreByValue { get; private set; }
+
+        /** <inheritdoc /> */
+        public bool IsStatisticsEnabled { get; private set; }
+
+        /** <inheritdoc /> */
+        public bool IsManagementEnabled { get; private set; }
+
+        /** <inheritdoc /> */
+        public bool IsReadThrough { get; private set; }
+
+        /** <inheritdoc /> */
+        public bool IsWriteThrough { get; private set; }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs
new file mode 100644
index 0000000..3eb63ca
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Impl.Cache
+{
+    /// <summary>
+    /// Cache opcodes.
+    /// </summary>
+    internal enum CacheOp
+    {
+        Clear = 1,
+        ClearAll = 2,
+        ContainsKey = 3,
+        ContainsKeys = 4,
+        Get = 5,
+        GetAll = 6,
+        GetAndPut = 7,
+        GetAndPutIfAbsent = 8,
+        GetAndRemove = 9,
+        GetAndReplace = 10,
+        GetName = 11,
+        Invoke = 12,
+        InvokeAll = 13,
+        IsLocalLocked = 14,
+        LoadCache = 15,
+        LocEvict = 16,
+        LocLoadCache = 17,
+        LocPromote = 18,
+        LocalClear = 20,
+        LocalClearAll = 21,
+        Lock = 22,
+        LockAll = 23,
+        Metrics = 24,
+        Peek = 25,
+        Put = 26,
+        PutAll = 27,
+        PutIfAbsent = 28,
+        QryContinuous = 29,
+        QryScan = 30,
+        QrySql = 31,
+        QrySqlFields = 32,
+        QryTxt = 33,
+        RemoveAll = 34,
+        RemoveBool = 35,
+        RemoveObj = 36,
+        Replace2 = 37,
+        Replace3 = 38
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheProxyImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheProxyImpl.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheProxyImpl.cs
new file mode 100644
index 0000000..5c6ee07
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/CacheProxyImpl.cs
@@ -0,0 +1,499 @@
+/*
+ * 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.Cache
+{
+    using System.Collections;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Threading;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cache.Expiry;
+    using Apache.Ignite.Core.Cache.Query;
+    using Apache.Ignite.Core.Cache.Query.Continuous;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// Cache proxy.
+    /// </summary>
+    [SuppressMessage("Microsoft.Design", "CA1001:TypesThatOwnDisposableFieldsShouldBeDisposable")]
+    internal class CacheProxyImpl<TK, TV> : ICache<TK, TV>
+    {
+        /** wrapped cache instance */
+        private readonly CacheImpl<TK, TV> _cache;
+
+        /** */
+        private readonly ThreadLocal<int> _lastAsyncOp = new ThreadLocal<int>(() => PlatformTarget.OpNone);
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheProxyImpl{K, V}"/> class.
+        /// </summary>
+        /// <param name="cache">The cache to wrap.</param>
+        public CacheProxyImpl(CacheImpl<TK, TV> cache)
+        {
+            Debug.Assert(cache != null);
+
+            _cache = cache;
+        }
+
+        /** <inheritDoc /> */
+        public ICache<TK, TV> WithSkipStore()
+        {
+            return _cache.IsSkipStore ? this : new CacheProxyImpl<TK, TV>((CacheImpl<TK, TV>)_cache.WithSkipStore());
+        }
+
+        /** <inheritDoc /> */
+        public ICache<TK, TV> WithExpiryPolicy(IExpiryPolicy plc)
+        {
+            return new CacheProxyImpl<TK, TV>((CacheImpl<TK, TV>)_cache.WithExpiryPolicy(plc));
+        }
+
+        /** <inheritDoc /> */
+        public ICache<TK, TV> WithAsync()
+        {
+            return IsAsync ? this : new CacheProxyImpl<TK, TV>((CacheImpl<TK, TV>) _cache.WithAsync());
+        }
+
+        /** <inheritDoc /> */
+        public bool IsAsync
+        {
+            get { return _cache.IsAsync; }
+        }
+
+        /** <inheritDoc /> */
+        public IFuture GetFuture()
+        {
+            return GetFuture<object>();
+        }
+
+        /** <inheritDoc /> */
+        public IFuture<TResult> GetFuture<TResult>()
+        {
+            var fut = _cache.GetFuture<TResult>(_lastAsyncOp.Value);
+
+            ClearLastAsyncOp();
+
+            return fut;
+        }
+
+        /** <inheritDoc /> */
+        public IEnumerator<ICacheEntry<TK, TV>> GetEnumerator()
+        {
+            return _cache.GetEnumerator();
+        }
+
+        /** <inheritDoc /> */
+        IEnumerator IEnumerable.GetEnumerator()
+        {
+            return ((IEnumerable) _cache).GetEnumerator();
+        }
+
+        /** <inheritDoc /> */
+        public string Name
+        {
+            get { return _cache.Name; }
+        }
+
+        /** <inheritDoc /> */
+        public IIgnite Ignite
+        {
+            get { return _cache.Ignite; }
+        }
+
+        /** <inheritDoc /> */
+        public bool IsEmpty
+        {
+            get { return _cache.IsEmpty; }
+        }
+
+        /** <inheritDoc /> */
+        public bool KeepPortable
+        {
+            get { return _cache.KeepPortable; }
+        }
+
+        /// <summary>
+        /// Skip store flag.
+        /// </summary>
+        internal bool SkipStore
+        {
+            get { return _cache.IsSkipStore; }
+        }
+
+        /** <inheritDoc /> */
+        public ICache<TK1, TV1> WithKeepPortable<TK1, TV1>()
+        {
+            return new CacheProxyImpl<TK1, TV1>((CacheImpl<TK1, TV1>) _cache.WithKeepPortable<TK1, TV1>());
+        }
+
+        /** <inheritDoc /> */
+        public void LoadCache(ICacheEntryFilter<TK, TV> p, params object[] args)
+        {
+            _cache.LoadCache(p, args);
+
+            SetLastAsyncOp(CacheOp.LoadCache);
+        }
+
+        /** <inheritDoc /> */
+        public void LocalLoadCache(ICacheEntryFilter<TK, TV> p, params object[] args)
+        {
+            _cache.LocalLoadCache(p, args);
+
+            SetLastAsyncOp(CacheOp.LocLoadCache);
+        }
+
+        /** <inheritDoc /> */
+        public bool ContainsKey(TK key)
+        {
+            var result = _cache.ContainsKey(key);
+            
+            SetLastAsyncOp(CacheOp.ContainsKey);
+
+            return result;
+        }
+
+        /** <inheritDoc /> */
+        public bool ContainsKeys(IEnumerable<TK> keys)
+        {
+            var result = _cache.ContainsKeys(keys);
+
+            SetLastAsyncOp(CacheOp.ContainsKeys);
+
+            return result;
+        }
+
+        /** <inheritDoc /> */
+        public TV LocalPeek(TK key, params CachePeekMode[] modes)
+        {
+            return _cache.LocalPeek(key, modes);
+        }
+
+        /** <inheritDoc /> */
+        public TV Get(TK key)
+        {
+            var result = _cache.Get(key);
+            
+            SetLastAsyncOp(CacheOp.Get);
+
+            return result;
+        }
+
+        /** <inheritDoc /> */
+        public IDictionary<TK, TV> GetAll(IEnumerable<TK> keys)
+        {
+            var result = _cache.GetAll(keys);
+
+            SetLastAsyncOp(CacheOp.GetAll);
+
+            return result;
+        }
+
+        /** <inheritDoc /> */
+        public void Put(TK key, TV val)
+        {
+            _cache.Put(key, val);
+
+            SetLastAsyncOp(CacheOp.Put);
+        }
+
+        /** <inheritDoc /> */
+        public TV GetAndPut(TK key, TV val)
+        {
+            var result = _cache.GetAndPut(key, val);
+
+            SetLastAsyncOp(CacheOp.GetAndPut);
+
+            return result;
+        }
+
+        /** <inheritDoc /> */
+        public TV GetAndReplace(TK key, TV val)
+        {
+            var result = _cache.GetAndReplace(key, val);
+
+            SetLastAsyncOp(CacheOp.GetAndReplace);
+
+            return result;
+        }
+
+        /** <inheritDoc /> */
+        public TV GetAndRemove(TK key)
+        {
+            var result = _cache.GetAndRemove(key);
+
+            SetLastAsyncOp(CacheOp.GetAndRemove);
+
+            return result;
+        }
+
+        /** <inheritDoc /> */
+        public bool PutIfAbsent(TK key, TV val)
+        {
+            var result = _cache.PutIfAbsent(key, val);
+
+            SetLastAsyncOp(CacheOp.PutIfAbsent);
+
+            return result;
+        }
+
+        /** <inheritDoc /> */
+        public TV GetAndPutIfAbsent(TK key, TV val)
+        {
+            var result = _cache.GetAndPutIfAbsent(key, val);
+
+            SetLastAsyncOp(CacheOp.GetAndPutIfAbsent);
+
+            return result;
+        }
+
+        /** <inheritDoc /> */
+        public bool Replace(TK key, TV val)
+        {
+            var result = _cache.Replace(key, val);
+
+            SetLastAsyncOp(CacheOp.Replace2);
+
+            return result;
+        }
+
+        /** <inheritDoc /> */
+        public bool Replace(TK key, TV oldVal, TV newVal)
+        {
+            var result = _cache.Replace(key, oldVal, newVal);
+
+            SetLastAsyncOp(CacheOp.Replace3);
+
+            return result;
+        }
+
+        /** <inheritDoc /> */
+        public void PutAll(IDictionary<TK, TV> vals)
+        {
+            _cache.PutAll(vals);
+
+            SetLastAsyncOp(CacheOp.PutAll);
+        }
+
+        /** <inheritDoc /> */
+        public void LocalEvict(IEnumerable<TK> keys)
+        {
+            _cache.LocalEvict(keys);
+        }
+
+        /** <inheritDoc /> */
+        public void Clear()
+        {
+            _cache.Clear();
+
+            ClearLastAsyncOp();
+        }
+
+        /** <inheritDoc /> */
+        public void Clear(TK key)
+        {
+            _cache.Clear(key);
+
+            SetLastAsyncOp(CacheOp.Clear);
+        }
+
+        /** <inheritDoc /> */
+        public void ClearAll(IEnumerable<TK> keys)
+        {
+            _cache.ClearAll(keys);
+            
+            SetLastAsyncOp(CacheOp.ClearAll);
+        }
+
+        /** <inheritDoc /> */
+        public void LocalClear(TK key)
+        {
+            _cache.LocalClear(key);
+        }
+
+        /** <inheritDoc /> */
+        public void LocalClearAll(IEnumerable<TK> keys)
+        {
+            _cache.LocalClearAll(keys);
+        }
+
+        /** <inheritDoc /> */
+        public bool Remove(TK key)
+        {
+            var result = _cache.Remove(key);
+
+            SetLastAsyncOp(CacheOp.RemoveObj);
+
+            return result;
+        }
+
+        /** <inheritDoc /> */
+        public bool Remove(TK key, TV val)
+        {
+            var result = _cache.Remove(key, val);
+
+            SetLastAsyncOp(CacheOp.RemoveBool);
+
+            return result;
+        }
+
+        /** <inheritDoc /> */
+        public void RemoveAll(IEnumerable<TK> keys)
+        {
+            _cache.RemoveAll(keys);
+
+            SetLastAsyncOp(CacheOp.RemoveAll);
+        }
+
+        /** <inheritDoc /> */
+        public void RemoveAll()
+        {
+            _cache.RemoveAll();
+
+            ClearLastAsyncOp();
+        }
+
+        /** <inheritDoc /> */
+        public int LocalSize(params CachePeekMode[] modes)
+        {
+            return _cache.LocalSize(modes);
+        }
+
+        /** <inheritDoc /> */
+        public int Size(params CachePeekMode[] modes)
+        {
+            var result = _cache.Size(modes);
+
+            ClearLastAsyncOp();
+
+            return result;
+        }
+
+        /** <inheritDoc /> */
+        public void LocalPromote(IEnumerable<TK> keys)
+        {
+            _cache.LocalPromote(keys);
+        }
+
+        /** <inheritDoc /> */
+        public IQueryCursor<ICacheEntry<TK, TV>> Query(QueryBase qry)
+        {
+            return _cache.Query(qry);
+        }
+
+        /** <inheritDoc /> */
+        public IQueryCursor<IList> QueryFields(SqlFieldsQuery qry)
+        {
+            return _cache.QueryFields(qry);
+        }
+
+        /** <inheritDoc /> */
+        public IContinuousQueryHandle QueryContinuous(ContinuousQuery<TK, TV> qry)
+        {
+            return _cache.QueryContinuous(qry);
+        }
+
+        /** <inheritDoc /> */
+        public IContinuousQueryHandle<ICacheEntry<TK, TV>> QueryContinuous(ContinuousQuery<TK, TV> qry, QueryBase initialQry)
+        {
+            return _cache.QueryContinuous(qry, initialQry);
+        }
+
+        /** <inheritDoc /> */
+        public IEnumerable<ICacheEntry<TK, TV>> GetLocalEntries(params CachePeekMode[] peekModes)
+        {
+            return _cache.GetLocalEntries(peekModes);
+        }
+
+        /** <inheritDoc /> */
+        public TR Invoke<TR, TA>(TK key, ICacheEntryProcessor<TK, TV, TA, TR> processor, TA arg)
+        {
+            var result = _cache.Invoke(key, processor, arg);
+
+            SetLastAsyncOp(CacheOp.Invoke);
+
+            return result;
+        }
+
+        /** <inheritDoc /> */
+        public IDictionary<TK, ICacheEntryProcessorResult<TR>> InvokeAll<TR, TA>(IEnumerable<TK> keys,
+            ICacheEntryProcessor<TK, TV, TA, TR> processor, TA arg)
+        {
+            var result = _cache.InvokeAll(keys, processor, arg);
+
+            SetLastAsyncOp(CacheOp.InvokeAll);
+
+            return result;
+        }
+
+        /** <inheritDoc /> */
+        public ICacheLock Lock(TK key)
+        {
+            return _cache.Lock(key);
+        }
+
+        /** <inheritDoc /> */
+        public ICacheLock LockAll(IEnumerable<TK> keys)
+        {
+            return _cache.LockAll(keys);
+        }
+
+        /** <inheritDoc /> */
+        public bool IsLocalLocked(TK key, bool byCurrentThread)
+        {
+            return _cache.IsLocalLocked(key, byCurrentThread);
+        }
+
+        /** <inheritDoc /> */
+        public ICacheMetrics GetMetrics()
+        {
+            return _cache.GetMetrics();
+        }
+
+        /** <inheritDoc /> */
+        public IFuture Rebalance()
+        {
+            return _cache.Rebalance();
+        }
+
+        /** <inheritDoc /> */
+        public ICache<TK, TV> WithNoRetries()
+        {
+            return _cache.IsNoRetries ? this : new CacheProxyImpl<TK, TV>((CacheImpl<TK, TV>) _cache.WithNoRetries());
+        }
+
+        /// <summary>
+        /// Sets the last asynchronous op id.
+        /// </summary>
+        /// <param name="opId">The op identifier.</param>
+        private void SetLastAsyncOp(CacheOp opId)
+        {
+            if (IsAsync)
+                _lastAsyncOp.Value = (int) opId;
+        }
+
+        /// <summary>
+        /// Clears the last asynchronous op id.
+        /// This should be called in the end of each method that supports async and does not call SetLastAsyncOp.
+        /// </summary>
+        private void ClearLastAsyncOp()
+        {
+            if (IsAsync)
+                _lastAsyncOp.Value = PlatformTarget.OpNone;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Event/CacheEntryCreateEvent.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Event/CacheEntryCreateEvent.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Event/CacheEntryCreateEvent.cs
new file mode 100644
index 0000000..8d9dfef
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Event/CacheEntryCreateEvent.cs
@@ -0,0 +1,74 @@
+/*
+ * 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.Cache.Event
+{
+    using Apache.Ignite.Core.Cache.Event;
+
+    /// <summary>
+    /// Cache entry create event.
+    /// </summary>
+    internal class CacheEntryCreateEvent<TK, TV> : ICacheEntryEvent<TK, TV>
+    {
+        /** Key.*/
+        private readonly TK _key;
+
+        /** Value.*/
+        private readonly TV _val;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="key">Key.</param>
+        /// <param name="val">Value.</param>
+        public CacheEntryCreateEvent(TK key, TV val)
+        {
+            _key = key;
+            _val = val;
+        }
+
+        /** <inheritdoc /> */
+        public TK Key
+        {
+            get { return _key; }
+        }
+
+        /** <inheritdoc /> */
+        public TV Value
+        {
+            get { return _val; }
+        }
+
+        /** <inheritdoc /> */
+        public TV OldValue
+        {
+            get { return default(TV); }
+        }
+
+        /** <inheritdoc /> */
+        public bool HasOldValue
+        {
+            get { return false; }
+        }
+
+        /** <inheritdoc /> */
+        public CacheEntryEventType EventType
+        {
+            get { return CacheEntryEventType.Created; }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Event/CacheEntryRemoveEvent.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Event/CacheEntryRemoveEvent.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Event/CacheEntryRemoveEvent.cs
new file mode 100644
index 0000000..a44a800
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Event/CacheEntryRemoveEvent.cs
@@ -0,0 +1,74 @@
+/*
+ * 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.Cache.Event
+{
+    using Apache.Ignite.Core.Cache.Event;
+
+    /// <summary>
+    /// Cache entry remove event.
+    /// </summary>
+    internal class CacheEntryRemoveEvent<TK, TV> : ICacheEntryEvent<TK, TV>
+    {
+        /** Key.*/
+        private readonly TK _key;
+        
+        /** Old value.*/
+        private readonly TV _oldVal;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="key">Key.</param>
+        /// <param name="oldVal">Old value.</param>
+        public CacheEntryRemoveEvent(TK key, TV oldVal)
+        {
+            _key = key;
+            _oldVal = oldVal;
+        }
+
+        /** <inheritdoc /> */
+        public TK Key
+        {
+            get { return _key; }
+        }
+
+        /** <inheritdoc /> */
+        public TV Value
+        {
+            get { return default(TV); }
+        }
+
+        /** <inheritdoc /> */
+        public TV OldValue
+        {
+            get { return _oldVal; }
+        }
+
+        /** <inheritdoc /> */
+        public bool HasOldValue
+        {
+            get { return true; }
+        }
+
+        /** <inheritdoc /> */
+        public CacheEntryEventType EventType
+        {
+            get { return CacheEntryEventType.Removed; }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Event/CacheEntryUpdateEvent.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Event/CacheEntryUpdateEvent.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Event/CacheEntryUpdateEvent.cs
new file mode 100644
index 0000000..e6fb927
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Event/CacheEntryUpdateEvent.cs
@@ -0,0 +1,79 @@
+/*
+ * 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.Cache.Event
+{
+    using Apache.Ignite.Core.Cache.Event;
+
+    /// <summary>
+    /// Cache entry update event.
+    /// </summary>
+    internal class CacheEntryUpdateEvent<TK, TV> : ICacheEntryEvent<TK, TV>
+    {
+        /** Key.*/
+        private readonly TK _key;
+
+        /** Value.*/
+        private readonly TV _val;
+
+        /** Old value.*/
+        private readonly TV _oldVal;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="key">Key.</param>
+        /// <param name="oldVal">Old value.</param>
+        /// <param name="val">Value.</param>
+        public CacheEntryUpdateEvent(TK key, TV oldVal, TV val)
+        {
+            _key = key;
+            _oldVal = oldVal;
+            _val = val;
+        }
+
+        /** <inheritdoc /> */
+        public TK Key
+        {
+            get { return _key; }
+        }
+
+        /** <inheritdoc /> */
+        public TV Value
+        {
+            get { return _val; }
+        }
+
+        /** <inheritdoc /> */
+        public TV OldValue
+        {
+            get { return _oldVal; }
+        }
+
+        /** <inheritdoc /> */
+        public bool HasOldValue
+        {
+            get { return true; }
+        }
+
+        /** <inheritdoc /> */
+        public CacheEntryEventType EventType
+        {
+            get { return CacheEntryEventType.Updated; }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/MutableCacheEntry.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/MutableCacheEntry.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/MutableCacheEntry.cs
new file mode 100644
index 0000000..2c69043
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/MutableCacheEntry.cs
@@ -0,0 +1,163 @@
+/*
+ * 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.Cache
+{
+    using System;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Impl.Common;
+
+    /// <summary>
+    /// Represents a cache entry.
+    /// </summary>
+    internal class MutableCacheEntry<TK, TV> : IMutableCacheEntry<TK, TV>, IMutableCacheEntryInternal
+    {
+        // Entry value
+        private TV _value;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="MutableCacheEntry{K, V}"/> class.
+        /// </summary>
+        /// <param name="key">The key.</param>
+        public MutableCacheEntry(TK key)
+        {
+            Key = key;
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="MutableCacheEntry{K, V}"/> class.
+        /// </summary>
+        /// <param name="key">The key.</param>
+        /// <param name="value">The value.</param>
+        public MutableCacheEntry(TK key, TV value)
+        {
+            Key = key;
+            _value = value;
+            Exists = true;
+        }
+
+        /** <inheritdoc /> */
+        public TK Key { get; private set; }
+
+        /** <inheritdoc /> */
+        object IMutableCacheEntryInternal.Key
+        {
+            get { return Key; }
+        }
+
+        /** <inheritdoc /> */
+        public TV Value
+        {
+            get { return _value; }
+            set
+            {
+                _value = value;
+                Exists = true;
+                State = MutableCacheEntryState.ValueSet;
+            }
+        }
+
+        /** <inheritdoc /> */
+        object IMutableCacheEntryInternal.Value
+        {
+            get { return Value; }
+        }
+
+        /** <inheritdoc /> */
+        public bool Exists { get; private set; }
+
+        /** <inheritdoc /> */
+        public void Remove()
+        {
+            Value = default(TV);
+            Exists = false;
+            State = MutableCacheEntryState.Removed;
+        }
+
+        /** <inheritdoc /> */
+        public MutableCacheEntryState State { get; private set; }
+    }
+
+    /// <summary>
+    /// Internal non-generic representation of a mutable cache entry.
+    /// </summary>
+    internal interface IMutableCacheEntryInternal
+    {
+        /// <summary>
+        /// Gets the key.
+        /// </summary>
+        object Key { get; }
+
+        /// <summary>
+        /// Gets the value.
+        /// </summary>
+        object Value { get; }
+
+        /// <summary>
+        /// Gets a value indicating whether cache entry exists.
+        /// </summary>
+        bool Exists { get; }
+
+        /// <summary>
+        /// Gets the state indicating user operation on this instance.
+        /// </summary>
+        MutableCacheEntryState State { get; }
+    }
+
+    /// <summary>
+    /// Mutable cache entry factory.
+    /// </summary>
+    internal static class MutableCacheEntry
+    {
+        private static readonly CopyOnWriteConcurrentDictionary<Tuple<Type, Type>, Func<object, object, bool, IMutableCacheEntryInternal>> 
+            Ctors = new CopyOnWriteConcurrentDictionary<Tuple<Type, Type>, Func<object, object, bool, IMutableCacheEntryInternal>>();
+
+        public static Func<object, object, bool, IMutableCacheEntryInternal> GetCtor(Type keyType, Type valType)
+        {
+            Func<object, object, bool, IMutableCacheEntryInternal> result;
+            var funcKey = new Tuple<Type, Type>(keyType, valType);
+
+            return Ctors.TryGetValue(funcKey, out result)
+                ? result
+                : Ctors.GetOrAdd(funcKey, x =>
+                {
+                    var entryType = typeof (MutableCacheEntry<,>).MakeGenericType(keyType, valType);
+
+                    var oneArg = DelegateConverter.CompileCtor<Func<object, IMutableCacheEntryInternal>>(entryType,
+                        new[] {keyType}, false);
+
+                    var twoArg =
+                        DelegateConverter.CompileCtor<Func<object, object, IMutableCacheEntryInternal>>(entryType, 
+                        new[] {keyType, valType}, false);
+
+                    return (k, v, exists) => exists ? twoArg(k, v) : oneArg(k);
+                });
+        }
+    }
+
+    /// <summary>
+    /// Represents result of user operation on a mutable cache entry.
+    /// </summary>
+    internal enum MutableCacheEntryState : byte
+    {
+        Intact = 0,
+        ValueSet = 1,
+        Removed = 2,
+        ErrPortable = 3,
+        ErrString = 4
+    }
+}
\ No newline at end of file


[37/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/ICacheEntryEventListener.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/ICacheEntryEventListener.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/ICacheEntryEventListener.cs
new file mode 100644
index 0000000..76ae04c
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/ICacheEntryEventListener.cs
@@ -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.
+ */
+
+namespace Apache.Ignite.Core.Cache.Event
+{
+    using System.Collections.Generic;
+
+    /// <summary>
+    /// Cache entry event listener.
+    /// </summary>
+    public interface ICacheEntryEventListener<TK, TV>
+    {
+        /// <summary>
+        /// Event callback.
+        /// </summary>
+        /// <param name="evts">Events.</param>
+        void OnEvent(IEnumerable<ICacheEntryEvent<TK, TV>> evts);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Expiry/ExpiryPolicy.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Expiry/ExpiryPolicy.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Expiry/ExpiryPolicy.cs
new file mode 100644
index 0000000..1feccbd
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Expiry/ExpiryPolicy.cs
@@ -0,0 +1,89 @@
+/*
+ * 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.Cache.Expiry
+{
+    using System;
+
+    /// <summary>
+    /// Default expiry policy implementation with all durations deinfed explicitly.
+    /// </summary>
+    public class ExpiryPolicy : IExpiryPolicy
+    {
+        /** Expiry for create. */
+        private readonly TimeSpan? _create;
+
+        /** Expiry for update. */
+        private readonly TimeSpan? _update;
+
+        /** Expiry for access. */
+        private readonly TimeSpan? _access;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="create">Expiry for create.</param>
+        /// <param name="update">Expiry for udpate.</param>
+        /// <param name="access">Expiry for access.</param>
+        public ExpiryPolicy(TimeSpan? create, TimeSpan? update, TimeSpan? access)
+        {
+            _create = create;
+            _update = update;
+            _access = access;
+        }
+
+        /// <summary>
+        /// Gets expiry for create operation.
+        /// <para />
+        /// If <c>TimeSpan.ZERO</c> is returned, cache entry is considered immediately expired
+        /// and will not be added to cache. 
+        /// <para />
+        /// If <c>null</c> is returned, no change to previously understood expiry is performed.
+        /// </summary>
+        /// <returns>Expiry for create opeartion.</returns>
+        public TimeSpan? GetExpiryForCreate()
+        {
+            return _create;
+        }
+
+        /// <summary>
+        /// Gets expiry for update operation.
+        /// <para />
+        /// If <c>TimeSpan.ZERO</c> is returned, cache entry is considered immediately expired.
+        /// <para />
+        /// If <c>null</c> is returned, no change to previously understood expiry is performed.
+        /// </summary>
+        /// <returns>Expiry for update operation.</returns>
+        public TimeSpan? GetExpiryForUpdate()
+        {
+            return _update;
+        }
+
+        /// <summary>
+        /// Gets expiry for access operation.
+        /// <para />
+        /// If <c>TimeSpan.ZERO</c> is returned, cache entry is considered immediately expired.
+        /// <para />
+        /// If <c>null</c> is returned, no change to previously understood expiry is performed.
+        /// </summary>
+        /// <returns>Expiry for access operation.</returns>
+        public TimeSpan? GetExpiryForAccess()
+        {
+            return _access;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Expiry/IExpiryPolicy.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Expiry/IExpiryPolicy.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Expiry/IExpiryPolicy.cs
new file mode 100644
index 0000000..ff627ae
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Expiry/IExpiryPolicy.cs
@@ -0,0 +1,59 @@
+/*
+ * 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.Cache.Expiry
+{
+    using System;
+
+    /// <summary>
+    /// Defines functions to determine when cache entries will expire based on
+    /// creation, access and modification operations.
+    /// </summary>
+    public interface IExpiryPolicy
+    {
+        /// <summary>
+        /// Gets expiry for create operation.
+        /// <para />
+        /// If <c>TimeSpan.ZERO</c> is returned, cache entry is considered immediately expired
+        /// and will not be added to cache. 
+        /// <para />
+        /// If <c>null</c> is returned, no change to previously understood expiry is performed.
+        /// </summary>
+        /// <returns>Expiry for create opeartion.</returns>
+        TimeSpan? GetExpiryForCreate();
+
+        /// <summary>
+        /// Gets expiry for update operation.
+        /// <para />
+        /// If <c>TimeSpan.ZERO</c> is returned, cache entry is considered immediately expired.
+        /// <para />
+        /// If <c>null</c> is returned, no change to previously understood expiry is performed.
+        /// </summary>
+        /// <returns>Expiry for update operation.</returns>
+        TimeSpan? GetExpiryForUpdate();
+
+        /// <summary>
+        /// Gets expiry for access operation.
+        /// <para />
+        /// If <c>TimeSpan.ZERO</c> is returned, cache entry is considered immediately expired.
+        /// <para />
+        /// If <c>null</c> is returned, no change to previously understood expiry is performed.
+        /// </summary>
+        /// <returns>Expiry for access operation.</returns>
+        TimeSpan? GetExpiryForAccess();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICache.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICache.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICache.cs
new file mode 100644
index 0000000..3ee812a
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICache.cs
@@ -0,0 +1,542 @@
+/*
+ * 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.Cache
+{
+    using System;
+    using System.Collections;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Cache.Expiry;
+    using Apache.Ignite.Core.Cache.Query;
+    using Apache.Ignite.Core.Cache.Query.Continuous;
+    using Apache.Ignite.Core.Cache.Store;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Transactions;
+
+    /// <summary>
+    /// Main entry point for Ignite cache APIs. You can get a named cache by calling
+    /// <see cref="IIgnite.Cache{K, V}(string)"/> method.
+    /// <para />
+    /// Cache API supports distributed transactions. All <c>Get(...)</c>, <c>Put(...)</c>, <c>Replace(...)</c>,
+    /// and <c>Remove(...)</c> operations are transactional and will participate in an ongoing transaction,
+    /// if any. Other methods like <c>Peek(...)</c> or various <c>Contains(...)</c> methods may
+    /// be transaction-aware, i.e. check in-transaction entries first, but will not affect the current
+    /// state of transaction. See <see cref="ITransaction"/> documentation for more information
+    /// about transactions.
+    /// <para />
+    /// Neither <c>null</c> keys or values are allowed to be stored in cache. If a <c>null</c> value
+    /// happens to be in cache (e.g. after invalidation or remove), then cache will treat this case
+    /// as there is no value at all.
+    /// <para />
+    /// Note that cache is generic and you can only work with provided key and value types. If cache also
+    /// contains keys or values of other types, any attempt to retrieve them will result in
+    /// <see cref="InvalidCastException"/>. Use <see cref="ICache{Object, Object}"/> in order to work with entries
+    /// of arbitrary types.
+    /// <para/>
+    /// All members are thread-safe and may be used concurrently from multiple threads.
+    /// </summary>
+    /// <typeparam name="TK">Key type.</typeparam>
+    /// <typeparam name="TV">Value type.</typeparam>
+    public interface ICache<TK, TV> : IAsyncSupport<ICache<TK, TV>>, IEnumerable<ICacheEntry<TK, TV>>
+    {
+        /// <summary>
+        /// Name of this cache (<c>null</c> for default cache).
+        /// </summary>
+        string Name { get; }
+
+        /// <summary>
+        /// Ignite hosting this cache.
+        /// </summary>
+        IIgnite Ignite { get; }
+
+        /// <summary>
+        /// Checks whether this cache contains no key-value mappings.
+        /// <para />
+        /// Semantically equals to <c>ICache.Size(CachePeekMode.PRIMARY) == 0</c>.
+        /// </summary>
+        bool IsEmpty { get; }
+
+        /// <summary>
+        /// Gets a value indicating whether to keep values in portable form.
+        /// </summary>
+        bool KeepPortable { get; }
+
+        /// <summary>
+        /// Get another cache instance with read-through and write-through behavior disabled.
+        /// </summary>
+        /// <returns>Cache with read-through and write-through behavior disabled.</returns>
+        ICache<TK, TV> WithSkipStore();
+
+        /// <summary>
+        /// Returns cache with the specified expired policy set. This policy will be used for each operation
+        /// invoked on the returned cache.
+        /// <para />
+        /// Expiry durations for each operation are calculated only once and then used as constants. Please
+        /// consider this when implementing customg expiry policy implementations.
+        /// </summary>
+        /// <param name="plc">Expiry policy to use.</param>
+        /// <returns>Cache instance with the specified expiry policy set.</returns>
+        ICache<TK, TV> WithExpiryPolicy(IExpiryPolicy plc);
+
+        /// <summary>
+        /// Gets cache with KeepPortable mode enabled, changing key and/or value types if necessary.
+        /// You can only change key/value types when transitioning from non-portable to portable cache;
+        /// Changing type of portable cache is not allowed and will throw an <see cref="InvalidOperationException"/>
+        /// </summary>
+        /// <typeparam name="TK1">Key type in portable mode.</typeparam>
+        /// <typeparam name="TV1">Value type in protable mode.</typeparam>
+        /// <returns>Cache instance with portable mode enabled.</returns>
+        ICache<TK1, TV1> WithKeepPortable<TK1, TV1>();
+
+        /// <summary>
+        /// Executes <see cref="LocalLoadCache"/> on all cache nodes.
+        /// </summary>
+        /// <param name="p">
+        /// Optional predicate. If provided, will be used to filter values to be put into cache.
+        /// </param>
+        /// <param name="args">
+        /// Optional user arguments to be passed into <see cref="ICacheStore.LoadCache" />.
+        /// </param>
+        [AsyncSupported]
+        void LoadCache(ICacheEntryFilter<TK, TV> p, params object[] args);
+
+        /// <summary>
+        /// Delegates to <see cref="ICacheStore.LoadCache" /> method to load state 
+        /// from the underlying persistent storage. The loaded values will then be given 
+        /// to the optionally passed in predicate, and, if the predicate returns true, 
+        /// will be stored in cache. If predicate is null, then all loaded values will be stored in cache.
+        /// </summary>
+        /// <param name="p">
+        /// Optional predicate. If provided, will be used to filter values to be put into cache.
+        /// </param>
+        /// <param name="args">
+        /// Optional user arguments to be passed into <see cref="ICacheStore.LoadCache" />.
+        /// </param>
+        [AsyncSupported]
+        void LocalLoadCache(ICacheEntryFilter<TK, TV> p, params object[] args);
+
+        /// <summary>
+        /// Check if cache contains mapping for this key.
+        /// </summary>
+        /// <param name="key">Key.</param>
+        /// <returns>True if cache contains mapping for this key.</returns>
+        [AsyncSupported]
+        bool ContainsKey(TK key);
+
+        /// <summary>
+        /// Check if cache contains mapping for these keys.
+        /// </summary>
+        /// <param name="keys">Keys.</param>
+        /// <returns>True if cache contains mapping for all these keys.</returns>
+        [AsyncSupported]
+        bool ContainsKeys(IEnumerable<TK> keys);
+
+        /// <summary>
+        /// Peeks at cached value using optional set of peek modes. This method will sequentially
+        /// iterate over given peek modes, and try to peek at value using each peek mode. Once a
+        /// non-null value is found, it will be immediately returned.
+        /// This method does not participate in any transactions, however, it may peek at transactional
+        /// value depending on the peek modes used.
+        /// </summary>
+        /// <param name="key">Key.</param>
+        /// <param name="modes">Peek modes.</param>
+        /// <returns>Peeked value.</returns>
+        TV LocalPeek(TK key, params CachePeekMode[] modes);
+
+        /// <summary>
+        /// Retrieves value mapped to the specified key from cache.
+        /// If the value is not present in cache, then it will be looked up from swap storage. If
+        /// it's not present in swap, or if swap is disable, and if read-through is allowed, value
+        /// will be loaded from persistent store.
+        /// This method is transactional and will enlist the entry into ongoing transaction if there is one.
+        /// </summary>
+        /// <param name="key">Key.</param>
+        /// <returns>Value.</returns>
+        [AsyncSupported]
+        TV Get(TK key);
+
+        /// <summary>
+        /// Retrieves values mapped to the specified keys from cache.
+        /// If some value is not present in cache, then it will be looked up from swap storage. If
+        /// it's not present in swap, or if swap is disabled, and if read-through is allowed, value
+        /// will be loaded from persistent store.
+        /// This method is transactional and will enlist the entry into ongoing transaction if there is one.
+        /// </summary>
+        /// <param name="keys">Keys.</param>
+        /// <returns>Map of key-value pairs.</returns>
+        [AsyncSupported]
+        IDictionary<TK, TV> GetAll(IEnumerable<TK> keys);
+
+        /// <summary>
+        /// Associates the specified value with the specified key in the cache.
+        /// <para />
+        /// If the cache previously contained a mapping for the key, 
+        /// the old value is replaced by the specified value.
+        /// </summary>
+        /// <param name="key">Key with which the specified value is to be associated.</param>
+        /// <param name="val">Value to be associated with the specified key.</param>
+        [AsyncSupported]
+        void Put(TK key, TV val);
+
+        /// <summary>
+        /// Associates the specified value with the specified key in this cache,
+        /// returning an existing value if one existed.
+        /// </summary>
+        /// <param name="key">Key with which the specified value is to be associated.</param>
+        /// <param name="val">Value to be associated with the specified key.</param>
+        /// <returns>
+        /// The value associated with the key at the start of the operation or null if none was associated.
+        /// </returns>
+        [AsyncSupported]
+        TV GetAndPut(TK key, TV val);
+        
+        /// <summary>
+        /// Atomically replaces the value for a given key if and only if there is a value currently mapped by the key.
+        /// </summary>
+        /// <param name="key">Key with which the specified value is to be associated.</param>
+        /// <param name="val">Value to be associated with the specified key.</param>
+        /// <returns>
+        /// The previous value associated with the specified key, or null if there was no mapping for the key.
+        /// </returns>
+        [AsyncSupported]
+        TV GetAndReplace(TK key, TV val);
+
+        /// <summary>
+        /// Atomically removes the entry for a key only if currently mapped to some value.
+        /// </summary>
+        /// <param name="key">Key with which the specified value is associated.</param>
+        /// <returns>The value if one existed or null if no mapping existed for this key.</returns>
+        [AsyncSupported]
+        TV GetAndRemove(TK key);
+
+        /// <summary>
+        /// Atomically associates the specified key with the given value if it is not already associated with a value.
+        /// </summary>
+        /// <param name="key">Key with which the specified value is to be associated.</param>
+        /// <param name="val">Value to be associated with the specified key.</param>
+        /// <returns>True if a value was set.</returns>
+        [AsyncSupported]
+        bool PutIfAbsent(TK key, TV val);
+
+        /// <summary>
+        /// Stores given key-value pair in cache only if cache had no previous mapping for it.
+        /// If cache previously contained value for the given key, then this value is returned.
+        /// In case of PARTITIONED or REPLICATED caches, the value will be loaded from the primary node,
+        /// which in its turn may load the value from the swap storage, and consecutively, if it's not
+        /// in swap, from the underlying persistent storage.
+        /// If the returned value is not needed, method putxIfAbsent() should be used instead of this one to
+        /// avoid the overhead associated with returning of the previous value.
+        /// If write-through is enabled, the stored value will be persisted to store.
+        /// This method is transactional and will enlist the entry into ongoing transaction if there is one.
+        /// </summary>
+        /// <param name="key">Key to store in cache.</param>
+        /// <param name="val">Value to be associated with the given key.</param>
+        /// <returns>
+        /// Previously contained value regardless of whether put happened or not (null if there was no previous value).
+        /// </returns>
+        [AsyncSupported]
+        TV GetAndPutIfAbsent(TK key, TV val);
+
+        /// <summary>
+        /// Stores given key-value pair in cache only if there is a previous mapping for it.
+        /// If cache previously contained value for the given key, then this value is returned.
+        /// In case of PARTITIONED or REPLICATED caches, the value will be loaded from the primary node,
+        /// which in its turn may load the value from the swap storage, and consecutively, if it's not
+        /// in swap, rom the underlying persistent storage.
+        /// If write-through is enabled, the stored value will be persisted to store.
+        /// This method is transactional and will enlist the entry into ongoing transaction if there is one.
+        /// </summary>
+        /// <param name="key">Key to store in cache.</param>
+        /// <param name="val">Value to be associated with the given key.</param>
+        /// <returns>True if the value was replaced.</returns>
+        [AsyncSupported]
+        bool Replace(TK key, TV val);
+
+        /// <summary>
+        /// Stores given key-value pair in cache only if only if the previous value is equal to the
+        /// old value passed as argument.
+        /// This method is transactional and will enlist the entry into ongoing transaction if there is one.
+        /// </summary>
+        /// <param name="key">Key to store in cache.</param>
+        /// <param name="oldVal">Old value to match.</param>
+        /// <param name="newVal">Value to be associated with the given key.</param>
+        /// <returns>True if replace happened, false otherwise.</returns>
+        [AsyncSupported]
+        bool Replace(TK key, TV oldVal, TV newVal);
+
+        /// <summary>
+        /// Stores given key-value pairs in cache.
+        /// If write-through is enabled, the stored values will be persisted to store.
+        /// This method is transactional and will enlist the entry into ongoing transaction if there is one.
+        /// </summary>
+        /// <param name="vals">Key-value pairs to store in cache.</param>
+        [AsyncSupported]
+        void PutAll(IDictionary<TK, TV> vals);
+
+        /// <summary>
+        /// Attempts to evict all entries associated with keys. Note, that entry will be evicted only 
+        /// if it's not used (not participating in any locks or transactions).
+        /// </summary>
+        /// <param name="keys">Keys to evict from cache.</param>
+        void LocalEvict(IEnumerable<TK> keys);
+
+        /// <summary>
+        /// Clears the contents of the cache, without notifying listeners or CacheWriters.
+        /// </summary>
+        [AsyncSupported]
+        void Clear();
+
+        /// <summary>
+        /// Clear entry from the cache and swap storage, without notifying listeners or CacheWriters.
+        /// Entry is cleared only if it is not currently locked, and is not participating in a transaction.
+        /// </summary>
+        /// <param name="key">Key to clear.</param>
+        [AsyncSupported]
+        void Clear(TK key);
+
+        /// <summary>
+        /// Clear entries from the cache and swap storage, without notifying listeners or CacheWriters.
+        /// Entry is cleared only if it is not currently locked, and is not participating in a transaction.
+        /// </summary>
+        /// <param name="keys">Keys to clear.</param>
+        [AsyncSupported]
+        void ClearAll(IEnumerable<TK> keys);
+
+        /// <summary>
+        /// Clear entry from the cache and swap storage, without notifying listeners or CacheWriters.
+        /// Entry is cleared only if it is not currently locked, and is not participating in a transaction.
+        /// <para />
+        /// Note that this operation is local as it merely clears
+        /// an entry from local cache, it does not remove entries from remote caches.
+        /// </summary>
+        /// <param name="key">Key to clear.</param>
+        void LocalClear(TK key);
+
+        /// <summary>
+        /// Clear entries from the cache and swap storage, without notifying listeners or CacheWriters.
+        /// Entry is cleared only if it is not currently locked, and is not participating in a transaction.
+        /// <para />
+        /// Note that this operation is local as it merely clears
+        /// entries from local cache, it does not remove entries from remote caches.
+        /// </summary>
+        /// <param name="keys">Keys to clear.</param>
+        void LocalClearAll(IEnumerable<TK> keys);
+
+        /// <summary>
+        /// Removes given key mapping from cache. If cache previously contained value for the given key,
+        /// then this value is returned. In case of PARTITIONED or REPLICATED caches, the value will be
+        /// loaded from the primary node, which in its turn may load the value from the disk-based swap
+        /// storage, and consecutively, if it's not in swap, from the underlying persistent storage.
+        /// If the returned value is not needed, method removex() should always be used instead of this
+        /// one to avoid the overhead associated with returning of the previous value.
+        /// If write-through is enabled, the value will be removed from store.
+        /// This method is transactional and will enlist the entry into ongoing transaction if there is one.
+        /// </summary>
+        /// <param name="key">Key whose mapping is to be removed from cache.</param>
+        /// <returns>False if there was no matching key.</returns>
+        [AsyncSupported]
+        bool Remove(TK key);
+
+        /// <summary>
+        /// Removes given key mapping from cache if one exists and value is equal to the passed in value.
+        /// If write-through is enabled, the value will be removed from store.
+        /// This method is transactional and will enlist the entry into ongoing transaction if there is one.
+        /// </summary>
+        /// <param name="key">Key whose mapping is to be removed from cache.</param>
+        /// <param name="val">Value to match against currently cached value.</param>
+        /// <returns>True if entry was removed, false otherwise.</returns>
+        [AsyncSupported]
+        bool Remove(TK key, TV val);
+
+        /// <summary>
+        /// Removes given key mappings from cache.
+        /// If write-through is enabled, the value will be removed from store.
+        /// This method is transactional and will enlist the entry into ongoing transaction if there is one.
+        /// </summary>
+        /// <param name="keys">Keys whose mappings are to be removed from cache.</param>
+        [AsyncSupported]
+        void RemoveAll(IEnumerable<TK> keys);
+
+        /// <summary>
+        /// Removes all mappings from cache.
+        /// If write-through is enabled, the value will be removed from store.
+        /// This method is transactional and will enlist the entry into ongoing transaction if there is one.
+        /// </summary>
+        [AsyncSupported]
+        void RemoveAll();
+
+        /// <summary>
+        /// Gets the number of all entries cached on this node.
+        /// </summary>
+        /// <param name="modes">Optional peek modes. If not provided, then total cache size is returned.</param>
+        /// <returns>Cache size on this node.</returns>
+        int LocalSize(params CachePeekMode[] modes);
+
+        /// <summary>
+        /// Gets the number of all entries cached across all nodes.
+        /// <para />
+        /// NOTE: this operation is distributed and will query all participating nodes for their cache sizes.
+        /// </summary>
+        /// <param name="modes">Optional peek modes. If not provided, then total cache size is returned.</param>
+        /// <returns>Cache size across all nodes.</returns>
+        [AsyncSupported]
+        int Size(params CachePeekMode[] modes);
+
+        /// <summary>
+        /// This method unswaps cache entries by given keys, if any, from swap storage into memory.
+        /// </summary>
+        /// <param name="keys">Keys to promote entries for.</param>
+        void LocalPromote(IEnumerable<TK> keys);
+        
+        /// <summary>
+        /// Queries cache.
+        /// </summary>
+        /// <param name="qry">Query.</param>
+        /// <returns>Cursor.</returns>
+        IQueryCursor<ICacheEntry<TK, TV>> Query(QueryBase qry);
+
+        /// <summary>
+        /// Queries separate entry fields.
+        /// </summary>
+        /// <param name="qry">SQL fields query.</param>
+        /// <returns>Cursor.</returns>
+        IQueryCursor<IList> QueryFields(SqlFieldsQuery qry);
+
+        /// <summary>
+        /// Start continuous query execution.
+        /// </summary>
+        /// <param name="qry">Continuous query.</param>
+        /// <returns>Handle to stop query execution.</returns>
+        IContinuousQueryHandle QueryContinuous(ContinuousQuery<TK, TV> qry);
+
+        /// <summary>
+        /// Start continuous query execution.
+        /// </summary>
+        /// <param name="qry">Continuous query.</param>
+        /// <param name="initialQry">
+        /// The initial query. This query will be executed before continuous listener is registered which allows 
+        /// to iterate through entries which have already existed at the time continuous query is executed.
+        /// </param>
+        /// <returns>
+        /// Handle to get initial query cursor or stop query execution.
+        /// </returns>
+        IContinuousQueryHandle<ICacheEntry<TK, TV>> QueryContinuous(ContinuousQuery<TK, TV> qry, QueryBase initialQry);
+        
+        /// <summary>
+        /// Get local cache entries.
+        /// </summary>
+        /// <param name="peekModes">Peek modes.</param>
+        /// <returns>Enumerable instance.</returns>
+        IEnumerable<ICacheEntry<TK, TV>> GetLocalEntries(params CachePeekMode[] peekModes);
+
+        /// <summary>
+        /// Invokes an <see cref="ICacheEntryProcessor{K, V, A, R}"/> against the 
+        /// <see cref="IMutableCacheEntry{K, V}"/> specified by the provided key. 
+        /// If an entry does not exist for the specified key, an attempt is made to load it (if a loader is configured) 
+        /// or a surrogate entry, consisting of the key with a null value is used instead.
+        /// </summary>
+        /// <typeparam name="TR">The type of the result.</typeparam>
+        /// <typeparam name="TA">The type of the argument.</typeparam>
+        /// <param name="key">The key.</param>
+        /// <param name="processor">The processor.</param>
+        /// <param name="arg">The argument.</param>
+        /// <returns>Result of the processing.</returns>
+        /// <exception cref="CacheEntryProcessorException">If an exception has occured during processing.</exception>
+        [AsyncSupported]
+        TR Invoke<TR, TA>(TK key, ICacheEntryProcessor<TK, TV, TA, TR> processor, TA arg);
+
+        /// <summary>
+        /// Invokes an <see cref="ICacheEntryProcessor{K, V, A, R}"/> against a set of keys.
+        /// If an entry does not exist for the specified key, an attempt is made to load it (if a loader is configured) 
+        /// or a surrogate entry, consisting of the key with a null value is used instead.
+        /// 
+        /// The order that the entries for the keys are processed is undefined. 
+        /// Implementations may choose to process the entries in any order, including concurrently.
+        /// Furthermore there is no guarantee implementations will use the same processor instance 
+        /// to process each entry, as the case may be in a non-local cache topology.
+        /// </summary>
+        /// <typeparam name="TR">The type of the result.</typeparam>
+        /// <typeparam name="TA">The type of the argument.</typeparam>
+        /// <param name="keys">The keys.</param>
+        /// <param name="processor">The processor.</param>
+        /// <param name="arg">The argument.</param>
+        /// <returns>
+        /// Map of <see cref="ICacheEntryProcessorResult{R}" /> of the processing per key, if any, 
+        /// defined by the <see cref="ICacheEntryProcessor{K,V,A,R}"/> implementation.  
+        /// No mappings will be returned for processors that return a null value for a key.
+        /// </returns>
+        /// <exception cref="CacheEntryProcessorException">If an exception has occured during processing.</exception>
+        [AsyncSupported]
+        IDictionary<TK, ICacheEntryProcessorResult<TR>> InvokeAll<TR, TA>(IEnumerable<TK> keys,
+            ICacheEntryProcessor<TK, TV, TA, TR> processor, TA arg);
+
+        /// <summary>
+        /// Creates an <see cref="ICacheLock"/> instance associated with passed key.
+        /// This method does not acquire lock immediately, you have to call appropriate method on returned instance.
+        /// </summary>
+        /// <param name="key">Key for lock.</param>
+        /// <returns>New <see cref="ICacheLock"/> instance associated with passed key.</returns>
+        ICacheLock Lock(TK key);
+
+        /// <summary>
+        /// Creates an <see cref="ICacheLock"/> instance associated with passed keys.
+        /// This method does not acquire lock immediately, you have to call appropriate method on returned instance.
+        /// </summary>
+        /// <param name="keys">Keys for lock.</param>
+        /// <returns>New <see cref="ICacheLock"/> instance associated with passed keys.</returns>
+        ICacheLock LockAll(IEnumerable<TK> keys);
+
+        /// <summary>
+        /// Checks if specified key is locked.
+        /// <para />
+        /// This is a local operation and does not involve any network trips
+        /// or access to persistent storage in any way.
+        /// </summary>
+        /// <param name="key">Key to check.</param>
+        /// <param name="byCurrentThread">
+        /// If true, checks that current thread owns a lock on this key; 
+        /// otherwise, checks that any thread on any node owns a lock on this key.
+        /// </param>
+        /// <returns>True if specified key is locked; otherwise, false.</returns>
+        bool IsLocalLocked(TK key, bool byCurrentThread);
+
+        /// <summary>
+        /// Gets snapshot metrics (statistics) for this cache.
+        /// </summary>
+        /// <returns>Cache metrics.</returns>
+        ICacheMetrics GetMetrics();
+
+        /// <summary>
+        /// Rebalances cache partitions. This method is usually used when rebalanceDelay configuration parameter 
+        /// has non-zero value. When many nodes are started or stopped almost concurrently, 
+        /// it is more efficient to delay rebalancing until the node topology is stable to make sure that no redundant 
+        /// re-partitioning happens.
+        /// <para />
+        /// In case of partitioned caches, for better efficiency user should usually make sure that new nodes get 
+        /// placed on the same place of consistent hash ring as the left nodes, and that nodes are restarted before
+        /// rebalanceDelay expires.
+        /// </summary>
+        /// <returns>Future that will be completed when rebalancing is finished.</returns>
+        IFuture Rebalance();
+
+        /// <summary>
+        /// Get another cache instance with no-retries behavior enabled.
+        /// </summary>
+        /// <returns>Cache with no-retries behavior enabled.</returns>
+        ICache<TK, TV> WithNoRetries();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheAffinity.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheAffinity.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheAffinity.cs
new file mode 100644
index 0000000..03a4e50
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheAffinity.cs
@@ -0,0 +1,161 @@
+/*
+ * 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.Cache
+{
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Cluster;
+
+    /// <summary>
+    /// Provides affinity information to detect which node is primary and which nodes are
+    /// backups for a partitioned cache. You can get an instance of this interface by calling
+    /// <see cref="IIgnite.Affinity(string)"/> method.
+    /// <para />
+    /// Mapping of a key to a node is a three-step operation. First step will get an affinity key for 
+    /// given key using <c>CacheAffinityKeyMapper</c>. If mapper is not specified, the original key 
+    /// will be used. Second step will map affinity key to partition using 
+    /// <c>CacheAffinityFunction.partition(Object)</c> method. Third step will map obtained partition 
+    /// to nodes for current grid topology version.
+    /// <para />
+    /// Interface provides various <c>mapKeysToNodes(...)</c> methods which provide node affinity mapping 
+    /// for given keys. All <c>mapKeysToNodes(...)</c> methods are not transactional and will not enlist
+    /// keys into ongoing transaction.
+    /// <para/>
+    /// All members are thread-safe and may be used concurrently from multiple threads.
+    /// </summary>
+    public interface ICacheAffinity
+    {
+        /// <summary>
+        /// Gets number of partitions in cache according to configured affinity function.
+        /// </summary>
+        /// <returns>Number of cache partitions.</returns>
+        int Partitions
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Gets partition id for the given key.
+        /// </summary>
+        /// <param name="key">Key to get partition id for.</param>
+        /// <returns>Partition id.</returns>
+        int Partition<TK>(TK key);
+
+        /// <summary>
+        /// Returns 'true' if given node is the primary node for given key.
+        /// </summary>
+        /// <param name="n">Node.</param>
+        /// <param name="key">Key.</param>
+        /// <returns>'True' if given node is the primary node for given key.</returns>
+        bool IsPrimary<TK>(IClusterNode n, TK key);
+
+        /// <summary>
+        /// Returns 'true' if given node is the backup node for given key.
+        /// </summary>
+        /// <param name="n">Node.</param>
+        /// <param name="key">Key.</param>
+        /// <returns>'True' if given node is the backup node for given key.</returns>
+        bool IsBackup<TK>(IClusterNode n, TK key);
+
+        /// <summary>
+        /// Returns 'true' if given node is either primary or backup node for given key.
+        /// </summary>
+        /// <param name="n">Node.</param>
+        /// <param name="key">Key.</param>
+        /// <returns>'True' if given node is either primary or backup node for given key.</returns>
+        bool IsPrimaryOrBackup<TK>(IClusterNode n, TK key);
+
+        /// <summary>
+        /// Gets partition ids for which nodes of the given projection has primary
+        /// ownership.
+        /// </summary>
+        /// <param name="n">Node.</param>
+        /// <returns>Partition ids for which given projection has primary ownership.</returns>
+        int[] PrimaryPartitions(IClusterNode n);
+
+        /// <summary>
+        /// Gets partition ids for which nodes of the given projection has backup
+        /// ownership.
+        /// </summary>
+        /// <param name="n">Node.</param>
+        /// <returns>Partition ids for which given projection has backup ownership.</returns>
+        int[] BackupPartitions(IClusterNode n);
+
+        /// <summary>
+        /// Gets partition ids for which nodes of the given projection has ownership
+        /// (either primary or backup).
+        /// </summary>
+        /// <param name="n">Node.</param>
+        /// <returns>Partition ids for which given projection has ownership.</returns>
+        int[] AllPartitions(IClusterNode n);
+
+        /// <summary>
+        /// Maps passed in key to a key which will be used for node affinity.
+        /// </summary>
+        /// <param name="key">Key to map.</param>
+        /// <returns>Key to be used for node-to-affinity mapping (may be the same key as passed in).</returns>
+        TR AffinityKey<TK, TR>(TK key);
+
+        /// <summary>
+        /// This method provides ability to detect which keys are mapped to which nodes.
+        /// Use it to determine which nodes are storing which keys prior to sending
+        /// jobs that access these keys.
+        /// </summary>
+        /// <param name="keys">Keys to map to nodes.</param>
+        /// <returns>Map of nodes to keys or empty map if there are no alive nodes for this cache.</returns>
+        IDictionary<IClusterNode, IList<TK>> MapKeysToNodes<TK>(IList<TK> keys);
+
+        /// <summary>
+        /// This method provides ability to detect to which primary node the given key
+        /// is mapped. Use it to determine which nodes are storing which keys prior to sending
+        /// jobs that access these keys.
+        /// </summary>
+        /// <param name="key">Keys to map to a node.</param>
+        /// <returns>Primary node for the key or null if there are no alive nodes for this cache.</returns>
+        IClusterNode MapKeyToNode<TK>(TK key);
+
+        /// <summary>
+        /// Gets primary and backup nodes for the key. Note that primary node is always
+        /// first in the returned collection.
+        /// </summary>
+        /// <param name="key"></param>
+        /// <returns></returns>
+        IList<IClusterNode> MapKeyToPrimaryAndBackups<TK>(TK key);
+
+        /// <summary>
+        /// Gets primary node for the given partition.
+        /// </summary>
+        /// <param name="part">Partition id.</param>
+        /// <returns>Primary node for the given partition.</returns>
+        IClusterNode MapPartitionToNode(int part);
+
+        /// <summary>
+        /// Gets primary nodes for the given partitions.
+        /// </summary>
+        /// <param name="parts">Partition ids.</param>
+        /// <returns>Mapping of given partitions to their primary nodes.</returns>
+        IDictionary<int, IClusterNode> MapPartitionsToNodes(IList<int> parts);
+
+        /// <summary>
+        /// Gets primary and backup nodes for partition. Note that primary node is always
+        /// first in the returned collection.
+        /// </summary>
+        /// <param name="part">Partition to get affinity nodes for.</param>
+        /// <returns>Collection of primary and backup nodes for partition with primary node always first</returns>
+        IList<IClusterNode> MapPartitionToPrimaryAndBackups(int part);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheEntry.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheEntry.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheEntry.cs
new file mode 100644
index 0000000..49ebfec
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheEntry.cs
@@ -0,0 +1,37 @@
+/*
+ * 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.Cache
+{
+    /// <summary>
+    /// Cache entry interface.
+    /// </summary>
+    /// <typeparam name="TK">Key type.</typeparam>
+    /// <typeparam name="TV">Value type.</typeparam>
+    public interface ICacheEntry<out TK, out TV>
+    {
+        /// <summary>
+        /// Gets the key.
+        /// </summary>
+        TK Key { get; }
+
+        /// <summary>
+        /// Gets the value.
+        /// </summary>
+        TV Value { get; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheEntryFilter.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheEntryFilter.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheEntryFilter.cs
new file mode 100644
index 0000000..9c7ee88
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheEntryFilter.cs
@@ -0,0 +1,34 @@
+/*
+ * 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.Cache
+{
+    /// <summary>
+    /// Cache entry predicate.
+    /// </summary>
+    /// <typeparam name="TK">Key type.</typeparam>
+    /// <typeparam name="TV">Value type.</typeparam>
+    public interface ICacheEntryFilter<in TK, in TV>
+    {
+        /// <summary>
+        /// Returns a value indicating whether provided cache entry satisfies this predicate.
+        /// </summary>
+        /// <param name="entry">Cache entry.</param>
+        /// <returns>Value indicating whether provided cache entry satisfies this predicate.</returns>
+        bool Invoke(ICacheEntry<TK, TV> entry);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheEntryProcessor.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheEntryProcessor.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheEntryProcessor.cs
new file mode 100644
index 0000000..c8614c0
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheEntryProcessor.cs
@@ -0,0 +1,45 @@
+/*
+ * 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.Cache
+{
+    /// <summary>
+    /// An invocable function that allows applications to perform compound operations
+    /// on a cache entry atomically, according the defined consistency of a cache.
+    /// <para />
+    /// Any cache entry mutations will not take effect until after
+    /// the <see cref="Process" /> method has completedS execution.
+    /// <para />
+    /// If an exception is thrown by an entry processor, a Caching Implementation
+    /// must wrap any exception thrown wrapped in an <see cref="CacheEntryProcessorException" />
+    /// If this occurs no mutations will be made to the cache entry.
+    /// </summary>
+    /// <typeparam name="TK">Key type.</typeparam>
+    /// <typeparam name="TV">Value type.</typeparam>
+    /// <typeparam name="TA">The type of the processor argument.</typeparam>
+    /// <typeparam name="TR">The type of the processor result.</typeparam>
+    public interface ICacheEntryProcessor<in TK, TV, in TA, out TR>
+    {
+        /// <summary>
+        /// Process an entry.
+        /// </summary>
+        /// <param name="entry">The entry to process.</param>
+        /// <param name="arg">The argument.</param>
+        /// <returns>Processing result.</returns>
+        TR Process(IMutableCacheEntry<TK, TV> entry, TA arg);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheEntryProcessorResult.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheEntryProcessorResult.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheEntryProcessorResult.cs
new file mode 100644
index 0000000..2d0f709
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheEntryProcessorResult.cs
@@ -0,0 +1,40 @@
+/*
+ * 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.Cache
+{
+    /// <summary>
+    /// Represents a result of processing <see cref="ICacheEntry{K, V}"/> 
+    /// by <see cref="ICacheEntryProcessor{K, V, A, R}"/>.
+    /// </summary>
+    /// <typeparam name="T">Processor result type.</typeparam>
+    public interface ICacheEntryProcessorResult<out T>
+    {
+        /// <summary>
+        /// Gets the result of processing an entry.
+        /// <para />
+        /// If an exception was thrown during the processing of an entry, 
+        /// either by the <see cref="ICacheEntryProcessor{K, V, A, R}"/> itself 
+        /// or by the Caching implementation, the exceptions will be wrapped and re-thrown as a 
+        /// <see cref="CacheEntryProcessorException"/> when calling this property.
+        /// </summary>
+        /// <value>
+        /// The result.
+        /// </value>
+        T Result { get; }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheLock.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheLock.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheLock.cs
new file mode 100644
index 0000000..a930961
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheLock.cs
@@ -0,0 +1,58 @@
+/*
+ * 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.Cache
+{
+    using System;
+    using System.Threading;
+
+    /// <summary>
+    /// Cache locking interface.
+    /// <para/>
+    /// All members are thread-safe and may be used concurrently from multiple threads.
+    /// </summary>
+    public interface ICacheLock : IDisposable
+    {
+        /// <summary>
+        /// Acquires an exclusive lock.
+        /// </summary>
+        void Enter();
+
+        /// <summary>
+        /// Acquires an exclusive lock only if it is free at the time of invocation.
+        /// </summary>
+        /// <returns>True if the current thread acquires the lock; otherwise, false.</returns>
+        bool TryEnter();
+
+        /// <summary>
+        /// Attempts, for the specified amount of time, to acquire an exclusive lock.
+        /// </summary>
+        /// <param name="timeout">
+        /// A <see cref="TimeSpan" /> representing the amount of time to wait for the lock. 
+        /// A value of –1 millisecond specifies an infinite wait.
+        /// </param>
+        /// <returns>True if the current thread acquires the lock; otherwise, false.</returns>
+        bool TryEnter(TimeSpan timeout);
+
+        /// <summary>
+        /// Releases an exclusive lock on the specified object.
+        /// <see cref="IDisposable.Dispose"/> does not call this method and will throw 
+        /// <see cref="SynchronizationLockException"/> if this lock is acquired.
+        /// </summary>
+        void Exit();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs
new file mode 100644
index 0000000..3405625
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs
@@ -0,0 +1,486 @@
+/*
+ * 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.Cache
+{
+    /// <summary>
+    /// Cache metrics used to obtain statistics on cache itself.
+    /// </summary>
+    public interface ICacheMetrics
+    {
+        /// <summary>
+        /// The number of get requests that were satisfied by the cache.
+        /// </summary>
+        /// <returns>
+        /// The number of hits
+        /// </returns>
+        long CacheHits { get; }
+
+        /// <summary>
+        /// This is a measure of cache efficiency.
+        /// </summary>
+        /// <returns>
+        /// The percentage of successful hits, as a decimal e.g 75.
+        /// </returns>
+        float CacheHitPercentage { get; }
+
+        /// <summary>
+        /// A miss is a get request that is not satisfied.
+        /// </summary>
+        /// <returns>
+        /// The number of misses
+        /// </returns>
+        long CacheMisses { get; }
+
+        /// <summary>
+        /// Returns the percentage of cache accesses that did not find a requested entry in the cache.
+        /// </summary>
+        /// <returns>
+        /// The percentage of accesses that failed to find anything.
+        /// </returns>
+        float CacheMissPercentage { get; }
+
+        /// <summary>
+        /// The total number of requests to the cache. This will be equal to the sum of the hits and misses.
+        /// </summary>
+        /// <returns>
+        /// The number of gets.
+        /// </returns>
+        long CacheGets { get; }
+
+        /// <summary>
+        /// The total number of puts to the cache.
+        /// </summary>
+        /// <returns>
+        /// The number of puts.
+        /// </returns>
+        long CachePuts { get; }
+
+        /// <summary>
+        /// The total number of removals from the cache. This does not include evictions, where the cache itself
+        /// initiates the removal to make space.
+        /// </summary>
+        /// <returns>
+        /// The number of removals.
+        /// </returns>
+        long CacheRemovals { get; }
+
+        /// <summary>
+        /// The total number of evictions from the cache. An eviction is a removal initiated by the cache itself 
+        /// to free up space. An eviction is not treated as a removal and does not appear in the removal counts.
+        /// </summary>
+        /// <returns>
+        /// The number of evictions.
+        /// </returns>
+        long CacheEvictions { get; }
+
+        /// <summary>
+        /// The mean time to execute gets.
+        /// </summary>
+        /// <returns>
+        /// The time in �s.
+        /// </returns>
+        float AverageGetTime { get; }
+
+        /// <summary>
+        /// The mean time to execute puts.
+        /// </summary>
+        /// <returns>
+        /// The time in �s.
+        /// </returns>
+        float AveragePutTime { get; }
+
+        /// <summary>
+        /// The mean time to execute removes.
+        /// </summary>
+        /// <returns>
+        /// The time in �s.
+        /// </returns>
+        float AverageRemoveTime { get; }
+
+        /// <summary>
+        /// The mean time to execute tx commit.
+        /// </summary>
+        /// <returns>
+        /// The time in �s.
+        /// </returns>
+        float AverageTxCommitTime { get; }
+
+        /// <summary>
+        /// The mean time to execute tx rollbacks.
+        /// </summary>
+        /// <returns>
+        /// Number of transaction rollbacks.
+        /// </returns>
+        float AverageTxRollbackTime { get; }
+
+        /// <summary>
+        /// Gets total number of transaction commits.
+        /// </summary>
+        /// <returns>
+        /// Number of transaction commits.
+        /// </returns>
+        long CacheTxCommits { get; }
+
+        /// <summary>
+        /// Gets total number of transaction rollbacks.
+        /// </summary>
+        /// <returns>
+        /// Number of transaction rollbacks.
+        /// </returns>
+        long CacheTxRollbacks { get; }
+
+        /// <summary>
+        /// Gets cache name.
+        /// </summary>
+        /// <returns>
+        /// Cache name.
+        /// </returns>
+        string CacheName { get; }
+
+        /// <summary>
+        /// Gets number of entries that was swapped to disk.
+        /// </summary>
+        /// <returns>
+        /// Number of entries that was swapped to disk.
+        /// </returns>
+        long OverflowSize { get; }
+
+        /// <summary>
+        /// Gets number of entries stored in off-heap memory.
+        /// </summary>
+        /// <returns>
+        /// Number of entries stored in off-heap memory.
+        /// </returns>
+        long OffHeapEntriesCount { get; }
+
+        /// <summary>
+        /// Gets memory size allocated in off-heap.
+        /// </summary>
+        /// <returns>
+        /// Memory size allocated in off-heap.
+        /// </returns>
+        long OffHeapAllocatedSize { get; }
+
+        /// <summary>
+        /// Gets number of non-null values in the cache.
+        /// </summary>
+        /// <returns>
+        /// Number of non-null values in the cache.
+        /// </returns>
+        int Size { get; }
+
+        /// <summary>
+        /// Gets number of keys in the cache, possibly with null values.
+        /// </summary>
+        /// <returns>
+        /// Number of keys in the cache.
+        /// </returns>
+        int KeySize { get; }
+
+        /// <summary>
+        /// Returns true if this cache is empty.
+        /// </summary>
+        /// <returns>
+        /// True if this cache is empty.
+        /// </returns>
+        bool IsEmpty { get; }
+
+        /// <summary>
+        /// Gets current size of evict queue used to batch up evictions.
+        /// </summary>
+        /// <returns>
+        /// Current size of evict queue.
+        /// </returns>
+        int DhtEvictQueueCurrentSize { get; }
+
+        /// <summary>
+        /// Gets transaction per-thread map size.
+        /// </summary>
+        /// <returns>
+        /// Thread map size.
+        /// </returns>
+        int TxThreadMapSize { get; }
+
+        /// <summary>
+        /// Gets transaction per-Xid map size.
+        /// </summary>
+        /// <returns>
+        /// Transaction per-Xid map size.
+        /// </returns>
+        int TxXidMapSize { get; }
+
+        /// <summary>
+        /// Gets committed transaction queue size.
+        /// </summary>
+        /// <returns>
+        /// Committed transaction queue size.
+        /// </returns>
+        int TxCommitQueueSize { get; }
+
+        /// <summary>
+        /// Gets prepared transaction queue size.
+        /// </summary>
+        /// <returns>
+        /// Prepared transaction queue size.
+        /// </returns>
+        int TxPrepareQueueSize { get; }
+
+        /// <summary>
+        /// Gets start version counts map size.
+        /// </summary>
+        /// <returns>
+        /// Start version counts map size.
+        /// </returns>
+        int TxStartVersionCountsSize { get; }
+
+        /// <summary>
+        /// Gets number of cached committed transaction IDs.
+        /// </summary>
+        /// <returns>
+        /// Number of cached committed transaction IDs.
+        /// </returns>
+        int TxCommittedVersionsSize { get; }
+
+        /// <summary>
+        /// Gets number of cached rolled back transaction IDs.
+        /// </summary>
+        /// <returns>
+        /// Number of cached rolled back transaction IDs.
+        /// </returns>
+        int TxRolledbackVersionsSize { get; }
+
+        /// <summary>
+        /// Gets transaction DHT per-thread map size.
+        /// </summary>
+        /// <returns>
+        /// DHT thread map size.
+        /// </returns>
+        int TxDhtThreadMapSize { get; }
+
+        /// <summary>
+        /// Gets transaction DHT per-Xid map size.
+        /// </summary>
+        /// <returns>
+        /// Transaction DHT per-Xid map size.
+        /// </returns>
+        int TxDhtXidMapSize { get; }
+
+        /// <summary>
+        /// Gets committed DHT transaction queue size.
+        /// </summary>
+        /// <returns>
+        /// Committed DHT transaction queue size.
+        /// </returns>
+        int TxDhtCommitQueueSize { get; }
+
+        /// <summary>
+        /// Gets prepared DHT transaction queue size.
+        /// </summary>
+        /// <returns>
+        /// Prepared DHT transaction queue size.
+        /// </returns>
+        int TxDhtPrepareQueueSize { get; }
+
+        /// <summary>
+        /// Gets DHT start version counts map size.
+        /// </summary>
+        /// <returns>
+        /// DHT start version counts map size.
+        /// </returns>
+        int TxDhtStartVersionCountsSize { get; }
+
+        /// <summary>
+        /// Gets number of cached committed DHT transaction IDs.
+        /// </summary>
+        /// <returns>
+        /// Number of cached committed DHT transaction IDs.
+        /// </returns>
+        int TxDhtCommittedVersionsSize { get; }
+
+        /// <summary>
+        /// Gets number of cached rolled back DHT transaction IDs.
+        /// </summary>
+        /// <returns>
+        /// Number of cached rolled back DHT transaction IDs.
+        /// </returns>
+        int TxDhtRolledbackVersionsSize { get; }
+
+        /// <summary>
+        /// Returns true if write-behind is enabled.
+        /// </summary>
+        /// <returns>
+        /// True if write-behind is enabled.
+        /// </returns>
+        bool IsWriteBehindEnabled { get; }
+
+        /// <summary>
+        /// Gets the maximum size of the write-behind buffer. When the count of unique keys in write buffer exceeds 
+        /// this value, the buffer is scheduled for write to the underlying store. 
+        /// <para /> 
+        /// If this value is 0, then flush is performed only on time-elapsing basis. 
+        /// </summary>
+        /// <returns>
+        /// Buffer size that triggers flush procedure.
+        /// </returns>
+        int WriteBehindFlushSize { get; }
+
+        /// <summary>
+        /// Gets the number of flush threads that will perform store update operations.
+        /// </summary>
+        /// <returns>
+        /// Count of worker threads.
+        /// </returns>
+        int WriteBehindFlushThreadCount { get; }
+
+        /// <summary>
+        /// Gets the cache flush frequency. All pending operations on the underlying store will be performed 
+        /// within time interval not less then this value. 
+        /// <para /> If this value is 0, then flush is performed only when buffer size exceeds flush size.
+        /// </summary>
+        /// <returns>
+        /// Flush frequency in milliseconds.
+        /// </returns>
+        long WriteBehindFlushFrequency { get; }
+
+        /// <summary>
+        /// Gets the maximum count of similar (put or remove) operations that can be grouped to a single batch.
+        /// </summary>
+        /// <returns>
+        /// Maximum size of batch.
+        /// </returns>
+        int WriteBehindStoreBatchSize { get; }
+
+        /// <summary>
+        /// Gets count of write buffer overflow events since initialization. 
+        /// Each overflow event causes the ongoing flush operation to be performed synchronously.
+        /// </summary>
+        /// <returns>
+        /// Count of cache overflow events since start.
+        /// </returns>
+        int WriteBehindTotalCriticalOverflowCount { get; }
+
+        /// <summary>
+        /// Gets count of write buffer overflow events in progress at the moment. 
+        /// Each overflow event causes the ongoing flush operation to be performed synchronously.
+        /// </summary>
+        /// <returns>
+        /// Count of cache overflow events since start.
+        /// </returns>
+        int WriteBehindCriticalOverflowCount { get; }
+
+        /// <summary>
+        /// Gets count of cache entries that are in a store-retry state. 
+        /// An entry is assigned a store-retry state when underlying store failed due some reason 
+        /// and cache has enough space to retain this entry till the next try.
+        /// </summary>
+        /// <returns>
+        /// Count of entries in store-retry state.
+        /// </returns>
+        int WriteBehindErrorRetryCount { get; }
+
+        /// <summary>
+        /// Gets count of entries that were processed by the write-behind store 
+        /// and have not been flushed to the underlying store yet.
+        /// </summary>
+        /// <returns>
+        /// Total count of entries in cache store internal buffer.
+        /// </returns>
+        int WriteBehindBufferSize { get; }
+
+        /// <summary>
+        /// Determines the required type of keys for this cache, if any.
+        /// </summary>
+        /// <returns>
+        /// The fully qualified class name of the key type, or "java.lang.Object" if the type is undefined.
+        /// </returns>
+        string KeyType { get; }
+
+        /// <summary>
+        /// Determines the required type of values for this cache, if any.
+        /// </summary>
+        /// <returns>
+        /// The fully qualified class name of the value type, or "java.lang.Object" if the type is undefined.
+        /// </returns>
+        string ValueType { get; }
+
+        /// <summary>
+        /// Whether storeByValue true or storeByReference false. When true, both keys and values are stored by value. 
+        /// <para /> 
+        /// When false, both keys and values are stored by reference. Caches stored by reference are capable of 
+        /// mutation by any threads holding the reference. 
+        /// The effects are: 
+        /// - if the key is mutated, then the key may not be retrievable or removable
+        /// - if the value is mutated, then all threads in the JVM can potentially observe those mutations, subject
+        /// to the normal Java Memory Model rules.
+        /// Storage by reference only applies to the local heap. 
+        /// If an entry is moved off heap it will need to be transformed into a representation. 
+        /// Any mutations that occur after transformation may not be reflected in the cache. 
+        /// <para /> 
+        /// When a cache is storeByValue, any mutation to the key or value does not affect the key of value 
+        /// stored in the cache. 
+        /// <para /> 
+        /// The default value is true.
+        /// </summary>
+        /// <returns>
+        /// True if the cache is store by value
+        /// </returns>
+        bool IsStoreByValue { get; }
+
+        /// <summary>
+        /// Checks whether statistics collection is enabled in this cache. 
+        /// <para /> 
+        /// The default value is false.
+        /// </summary>
+        /// <returns>
+        /// True if statistics collection is enabled
+        /// </returns>
+        bool IsStatisticsEnabled { get; }
+
+        /// <summary>
+        /// Checks whether management is enabled on this cache. 
+        /// <para /> 
+        /// The default value is false.
+        /// </summary>
+        /// <returns>
+        /// True if management is enabled
+        /// </returns>
+        bool IsManagementEnabled { get; }
+
+        /// <summary>
+        /// Determines if a cache should operate in read-through mode. 
+        /// <para /> 
+        /// The default value is false
+        /// </summary>
+        /// <returns>
+        /// True when a cache is in "read-through" mode.
+        /// </returns>
+        bool IsReadThrough { get; }
+
+        /// <summary>
+        /// Determines if a cache should operate in "write-through" mode. 
+        /// <para /> 
+        /// Will appropriately cause the configured CacheWriter to be invoked. 
+        /// <para /> 
+        /// The default value is false
+        /// </summary>
+        /// <returns>
+        /// True when a cache is in "write-through" mode.
+        /// </returns>
+        bool IsWriteThrough { get; }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/IMutableCacheEntry.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/IMutableCacheEntry.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/IMutableCacheEntry.cs
new file mode 100644
index 0000000..ae71be6
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/IMutableCacheEntry.cs
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Cache
+{
+    /// <summary>
+    /// Mutable representation of <see cref="ICacheEntry{K, V}"/>
+    /// </summary>
+    /// <typeparam name="TK">Key type.</typeparam>
+    /// <typeparam name="TV">Value type.</typeparam>
+    public interface IMutableCacheEntry<out TK, TV> : ICacheEntry<TK, TV>
+    {
+        /// <summary>
+        /// Gets a value indicating whether cache entry exists in cache.
+        /// </summary>
+        bool Exists { get; }
+
+        /// <summary>
+        /// Removes the entry from the Cache.
+        /// </summary>
+        void Remove();
+
+        /// <summary>
+        /// Gets, sets or replaces the value associated with the key.
+        /// <para />
+        /// If <see cref="Exists"/> is false and setter is called then a mapping is added to the cache 
+        /// visible once the EntryProcessor completes.
+        /// <para />
+        /// After setter invocation <see cref="Exists"/> will return true.
+        /// </summary>
+        new TV Value { get; set; }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/Continuous/ContinuousQuery.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/Continuous/ContinuousQuery.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/Continuous/ContinuousQuery.cs
new file mode 100644
index 0000000..8f297a2
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/Continuous/ContinuousQuery.cs
@@ -0,0 +1,170 @@
+/*
+ * 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.Cache.Query.Continuous
+{
+    using System;
+    using System.Diagnostics.CodeAnalysis;
+    using Apache.Ignite.Core.Cache.Event;
+
+    /// <summary>
+    /// API for configuring continuous cache queries.
+    /// <para />
+    /// Continuous queries allow to register a remote and a listener for cache update events. 
+    /// If an update event passes the filter, it will be sent to the node that executed the 
+    /// query and listener will be notified on that node.
+    /// <para />
+    /// Continuous query can either be executed on the whole topology or only on local node.
+    /// <para />
+    /// In case query is distributed and a new node joins, it will get the filter for the query 
+    /// during discovery process before it actually joins topology, so no updates will be missed.
+    /// <para />
+    /// To execute the query use method 
+    /// <see cref="ICache{K,V}.QueryContinuous(ContinuousQuery{K,V})"/>.
+    /// </summary>
+    public class ContinuousQuery<TK, TV>
+    {
+        /// <summary>
+        /// Default buffer size.
+        /// </summary>
+        [SuppressMessage("Microsoft.Design", "CA1000:DoNotDeclareStaticMembersOnGenericTypes")]
+        public const int DfltBufSize = 1;
+
+        /// <summary>
+        /// Default time interval.
+        /// </summary>
+        [SuppressMessage("ReSharper", "StaticMemberInGenericType")]
+        [SuppressMessage("Microsoft.Design", "CA1000:DoNotDeclareStaticMembersOnGenericTypes")]
+        public static readonly TimeSpan DfltTimeInterval = new TimeSpan(0);
+
+        /// <summary>
+        /// Default auto-unsubscribe flag value.
+        /// </summary>
+        [SuppressMessage("Microsoft.Design", "CA1000:DoNotDeclareStaticMembersOnGenericTypes")]
+        public const bool DfltAutoUnsubscribe = true;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="lsnr">Listener.</param>
+        public ContinuousQuery(ICacheEntryEventListener<TK, TV> lsnr) : this(lsnr, false)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="lsnr">Listener.</param>
+        /// <param name="loc">Whether query should be executed locally.</param>
+        public ContinuousQuery(ICacheEntryEventListener<TK, TV> lsnr, bool loc) : this(lsnr, null, loc)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="lsnr">Listener.</param>
+        /// <param name="filter">Filter.</param>
+        public ContinuousQuery(ICacheEntryEventListener<TK, TV> lsnr, ICacheEntryEventFilter<TK, TV> filter)
+            : this(lsnr, filter, false)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="lsnr">Listener.</param>
+        /// <param name="filter">Filter.</param>
+        /// <param name="loc">Whether query should be executed locally.</param>
+        public ContinuousQuery(ICacheEntryEventListener<TK, TV> lsnr, ICacheEntryEventFilter<TK, TV> filter, bool loc)
+        {
+            Listener = lsnr;
+            Filter = filter;
+            Local = loc;
+
+            BufferSize = DfltBufSize;
+            TimeInterval = DfltTimeInterval;
+            AutoUnsubscribe = DfltAutoUnsubscribe;
+        }
+
+        /// <summary>
+        /// Cache entry event listener. Invoked on the node where continuous query execution 
+        /// has been started.
+        /// </summary>
+        public ICacheEntryEventListener<TK, TV> Listener { get; set; }
+
+        /// <summary>
+        /// Optional cache entry filter. Invoked on a node where cache event occurred. If filter
+        /// returns <c>false</c>, then cache entry event will not be sent to a node where
+        /// continuous query has been started.
+        /// <para />
+        /// Must be either portable or serializable in case query is not local.
+        /// </summary>
+        public ICacheEntryEventFilter<TK, TV> Filter { get; set; }
+
+        /// <summary>
+        /// Buffer size. When a cache update happens, entry is first put into a buffer. 
+        /// Entries from buffer will be sent to the master node only if the buffer is 
+        /// full or time provided via <see cref="TimeInterval"/> is exceeded.
+        /// <para />
+        /// Defaults to <see cref="DfltBufSize"/>
+        /// </summary>
+        public int BufferSize { get; set; }
+
+        /// <summary>
+        /// Time interval. When a cache update happens, entry is first put into a buffer. 
+        /// Entries from buffer will be sent to the master node only if the buffer is full 
+        /// (its size can be provided via <see cref="BufferSize"/> property) or time provided 
+        /// via this method is exceeded.
+        /// <para />
+        /// Defaults to <c>0</c> which means that time check is disabled and entries will be 
+        /// sent only when buffer is full.
+        /// </summary>
+        public TimeSpan TimeInterval { get; set; }
+
+        /// <summary>
+        /// Automatic unsubscribe flag. This flag indicates that query filters on remote nodes 
+        /// should be automatically unregistered if master node (node that initiated the query) 
+        /// leaves topology. If this flag is <c>false</c>, filters will be unregistered only 
+        /// when the query is cancelled from master node, and won't ever be unregistered if 
+        /// master node leaves grid.
+        /// <para />
+        /// Defaults to <c>true</c>.
+        /// </summary>
+        public bool AutoUnsubscribe { get; set; }
+
+        /// <summary>
+        /// Local flag. When set query will be executed only on local node, so only local 
+        /// entries will be returned as query result.
+        /// <para />
+        /// Defaults to <c>false</c>.
+        /// </summary>
+        public bool Local { get; set; }
+
+        /// <summary>
+        /// Validate continuous query state.
+        /// </summary>
+        internal void Validate()
+        {
+            if (Listener == null)
+                throw new ArgumentException("Listener cannot be null.");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/Continuous/IContinuousQueryHandle.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/Continuous/IContinuousQueryHandle.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/Continuous/IContinuousQueryHandle.cs
new file mode 100644
index 0000000..0a6f154
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/Continuous/IContinuousQueryHandle.cs
@@ -0,0 +1,51 @@
+/*
+ * 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.Cache.Query.Continuous
+{
+    using System;
+    using System.Diagnostics.CodeAnalysis;
+
+    /// <summary>
+    /// Represents a continuous query handle.
+    /// </summary>
+    [SuppressMessage("Microsoft.Design", "CA1040:AvoidEmptyInterfaces")]
+    public interface IContinuousQueryHandle : IDisposable
+    {
+        // No-op.
+    }
+
+    /// <summary>
+    /// Represents a continuous query handle.
+    /// </summary>
+    /// <typeparam name="T">Type of the initial query cursor.</typeparam>
+    public interface IContinuousQueryHandle<T> : IContinuousQueryHandle
+    {
+        /// <summary>
+        /// Gets the cursor for initial query.
+        /// </summary>
+        [Obsolete("GetInitialQueryCursor() method should be used instead.")]
+        IQueryCursor<T> InitialQueryCursor { get; }
+
+        /// <summary>
+        /// Gets the cursor for initial query.
+        /// Can be called only once, throws exception on consequent calls.
+        /// </summary>
+        /// <returns>Initial query cursor.</returns>
+        IQueryCursor<T> GetInitialQueryCursor();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/IQueryCursor.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/IQueryCursor.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/IQueryCursor.cs
new file mode 100644
index 0000000..9745765
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/IQueryCursor.cs
@@ -0,0 +1,40 @@
+/*
+ * 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.Cache.Query
+{
+    using System;
+    using System.Collections.Generic;
+
+    /// <summary>
+    /// Query result cursor. Can be processed either in iterative mode, or by taking
+    /// all entries using <see cref="IQueryCursor{T}.GetAll()"/> method.
+    /// <para />
+    /// Note that you get enumerator or call <code>GetAll()</code> method only once during
+    /// cursor lifetime. Any further attempts to get enumerator or all entries will result 
+    /// in exception.
+    /// </summary>
+    public interface IQueryCursor<T> : IEnumerable<T>, IDisposable
+    {
+        /// <summary>
+        /// Gets all query results. Use this method when you know in advance that query 
+        /// result is relatively small and will not cause memory utilization issues.
+        /// </summary>
+        /// <returns>List containing all query results.</returns>
+        IList<T> GetAll();
+    }
+}


[28/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
new file mode 100644
index 0000000..bf11397
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
@@ -0,0 +1,832 @@
+/*
+ * 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.Datastream
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Threading;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Datastream;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+    using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils;
+
+    /// <summary>
+    /// Data streamer internal interface to get rid of generics.
+    /// </summary>
+    internal interface IDataStreamer
+    {
+        /// <summary>
+        /// Callback invoked on topology size change.
+        /// </summary>
+        /// <param name="topVer">New topology version.</param>
+        /// <param name="topSize">New topology size.</param>
+        void TopologyChange(long topVer, int topSize);
+    }
+
+    /// <summary>
+    /// Data streamer implementation.
+    /// </summary>
+    internal class DataStreamerImpl<TK, TV> : PlatformDisposableTarget, IDataStreamer, IDataStreamer<TK, TV>
+    {
+
+#pragma warning disable 0420
+
+        /** Policy: continue. */
+        internal const int PlcContinue = 0;
+
+        /** Policy: close. */
+        internal const int PlcClose = 1;
+
+        /** Policy: cancel and close. */
+        internal const int PlcCancelClose = 2;
+
+        /** Policy: flush. */
+        internal const int PlcFlush = 3;
+        
+        /** Operation: update. */
+        private const int OpUpdate = 1;
+        
+        /** Operation: set receiver. */
+        private const int OpReceiver = 2;
+        
+        /** Cache name. */
+        private readonly string _cacheName;
+
+        /** Lock. */
+        private readonly ReaderWriterLockSlim _rwLock = new ReaderWriterLockSlim();
+
+        /** Closed event. */
+        private readonly ManualResetEventSlim _closedEvt = new ManualResetEventSlim(false);
+
+        /** Close future. */
+        private readonly Future<object> _closeFut = new Future<object>();
+
+        /** GC handle to this streamer. */
+        private readonly long _hnd;
+                
+        /** Topology version. */
+        private long _topVer;
+
+        /** Topology size. */
+        private int _topSize;
+        
+        /** Buffer send size. */
+        private volatile int _bufSndSize;
+
+        /** Current data streamer batch. */
+        private volatile DataStreamerBatch<TK, TV> _batch;
+
+        /** Flusher. */
+        private readonly Flusher<TK, TV> _flusher;
+
+        /** Receiver. */
+        private volatile IStreamReceiver<TK, TV> _rcv;
+
+        /** Receiver handle. */
+        private long _rcvHnd;
+
+        /** Receiver portable mode. */
+        private readonly bool _keepPortable;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="target">Target.</param>
+        /// <param name="marsh">Marshaller.</param>
+        /// <param name="cacheName">Cache name.</param>
+        /// <param name="keepPortable">Portable flag.</param>
+        public DataStreamerImpl(IUnmanagedTarget target, PortableMarshaller marsh, string cacheName, bool keepPortable)
+            : base(target, marsh)
+        {
+            _cacheName = cacheName;
+            _keepPortable = keepPortable;
+
+            // Create empty batch.
+            _batch = new DataStreamerBatch<TK, TV>();
+
+            // Allocate GC handle so that this data streamer could be easily dereferenced from native code.
+            WeakReference thisRef = new WeakReference(this);
+
+            _hnd = marsh.Ignite.HandleRegistry.Allocate(thisRef);
+
+            // Start topology listening. This call will ensure that buffer size member is updated.
+            UU.DataStreamerListenTopology(target, _hnd);
+
+            // Membar to ensure fields initialization before leaving constructor.
+            Thread.MemoryBarrier();
+
+            // Start flusher after everything else is initialized.
+            _flusher = new Flusher<TK, TV>(thisRef);
+
+            _flusher.RunThread();
+        }
+
+        /** <inheritDoc /> */
+        public string CacheName
+        {
+            get { return _cacheName; }
+        }
+
+        /** <inheritDoc /> */
+        public bool AllowOverwrite
+        {
+            get
+            {
+                _rwLock.EnterReadLock();
+
+                try
+                {
+                    ThrowIfDisposed();
+
+                    return UU.DataStreamerAllowOverwriteGet(Target);
+                }
+                finally
+                {
+                    _rwLock.ExitReadLock();
+                }
+            }
+            set
+            {
+                _rwLock.EnterWriteLock();
+
+                try
+                {
+                    ThrowIfDisposed();
+
+                    UU.DataStreamerAllowOverwriteSet(Target, value);
+                }
+                finally
+                {
+                    _rwLock.ExitWriteLock();
+                }
+            }
+        }
+
+        /** <inheritDoc /> */
+        public bool SkipStore
+        {
+            get
+            {
+                _rwLock.EnterReadLock(); 
+                
+                try
+                {
+                    ThrowIfDisposed();
+
+                    return UU.DataStreamerSkipStoreGet(Target);
+                }
+                finally
+                {
+                    _rwLock.ExitReadLock();
+                }
+            }
+            set
+            {
+                _rwLock.EnterWriteLock(); 
+                
+                try
+                {
+                    ThrowIfDisposed();
+
+                    UU.DataStreamerSkipStoreSet(Target, value);
+                }
+                finally
+                {
+                    _rwLock.ExitWriteLock();
+                }
+            }
+        }
+
+        /** <inheritDoc /> */
+        public int PerNodeBufferSize
+        {
+            get
+            {
+                _rwLock.EnterReadLock(); 
+                
+                try
+                {
+                    ThrowIfDisposed();
+
+                    return UU.DataStreamerPerNodeBufferSizeGet(Target);
+                }
+                finally
+                {
+                    _rwLock.ExitReadLock();
+                }
+            }
+            set
+            {
+                _rwLock.EnterWriteLock(); 
+                
+                try
+                {
+                    ThrowIfDisposed();
+
+                    UU.DataStreamerPerNodeBufferSizeSet(Target, value);
+
+                    _bufSndSize = _topSize * value;
+                }
+                finally
+                {
+                    _rwLock.ExitWriteLock();
+                }
+            }
+        }
+
+        /** <inheritDoc /> */
+        public int PerNodeParallelOperations
+        {
+            get
+            {
+                _rwLock.EnterReadLock(); 
+                
+                try
+                {
+                    ThrowIfDisposed();
+
+                    return UU.DataStreamerPerNodeParallelOperationsGet(Target);
+                }
+                finally
+                {
+                    _rwLock.ExitReadLock();
+                }
+
+            }
+            set
+            {
+                _rwLock.EnterWriteLock(); 
+                
+                try
+                {
+                    ThrowIfDisposed();
+
+                    UU.DataStreamerPerNodeParallelOperationsSet(Target, value);
+                }
+                finally
+                {
+                    _rwLock.ExitWriteLock();
+                }
+
+            }
+        }
+
+        /** <inheritDoc /> */
+        public long AutoFlushFrequency
+        {
+            get
+            {
+                _rwLock.EnterReadLock(); 
+                
+                try
+                {
+                    ThrowIfDisposed();
+
+                    return _flusher.Frequency;
+                }
+                finally
+                {
+                    _rwLock.ExitReadLock();
+                }
+
+            }
+            set
+            {
+                _rwLock.EnterWriteLock(); 
+                
+                try
+                {
+                    ThrowIfDisposed();
+
+                    _flusher.Frequency = value;
+                }
+                finally
+                {
+                    _rwLock.ExitWriteLock();
+                }
+            }
+        }
+
+        /** <inheritDoc /> */
+        public IFuture Future
+        {
+            get
+            {
+                ThrowIfDisposed();
+
+                return _closeFut;
+            }
+        }
+
+        /** <inheritDoc /> */
+        public IStreamReceiver<TK, TV> Receiver
+        {
+            get
+            {
+                ThrowIfDisposed();
+
+                return _rcv;
+            }
+            set
+            {
+                IgniteArgumentCheck.NotNull(value, "value");
+
+                var handleRegistry = Marshaller.Ignite.HandleRegistry;
+
+                _rwLock.EnterWriteLock();
+
+                try
+                {
+                    ThrowIfDisposed();
+
+                    if (_rcv == value)
+                        return;
+
+                    var rcvHolder = new StreamReceiverHolder(value,
+                        (rec, grid, cache, stream, keepPortable) =>
+                            StreamReceiverHolder.InvokeReceiver((IStreamReceiver<TK, TV>) rec, grid, cache, stream,
+                                keepPortable));
+
+                    var rcvHnd0 = handleRegistry.Allocate(rcvHolder);
+
+                    try
+                    {
+                        DoOutOp(OpReceiver, w =>
+                        {
+                            w.WriteLong(rcvHnd0);
+
+                            w.WriteObject(rcvHolder);
+                        });
+                    }
+                    catch (Exception)
+                    {
+                        handleRegistry.Release(rcvHnd0);
+                        throw;
+                    }
+
+                    if (_rcv != null)
+                        handleRegistry.Release(_rcvHnd);
+
+                    _rcv = value;
+                    _rcvHnd = rcvHnd0;
+                }
+                finally
+                {
+                    _rwLock.ExitWriteLock();
+                }
+            }
+        }
+
+        /** <inheritDoc /> */
+        public IFuture AddData(TK key, TV val)
+        {
+            ThrowIfDisposed(); 
+            
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            return Add0(new DataStreamerEntry<TK, TV>(key, val), 1);
+        }
+
+        /** <inheritDoc /> */
+        public IFuture AddData(KeyValuePair<TK, TV> pair)
+        {
+            ThrowIfDisposed();
+
+            return Add0(new DataStreamerEntry<TK, TV>(pair.Key, pair.Value), 1);
+        }
+        
+        /** <inheritDoc /> */
+        public IFuture AddData(ICollection<KeyValuePair<TK, TV>> entries)
+        {
+            ThrowIfDisposed();
+
+            IgniteArgumentCheck.NotNull(entries, "entries");
+
+            return Add0(entries, entries.Count);
+        }
+
+        /** <inheritDoc /> */
+        public IFuture RemoveData(TK key)
+        {
+            ThrowIfDisposed();
+
+            IgniteArgumentCheck.NotNull(key, "key");
+
+            return Add0(new DataStreamerRemoveEntry<TK>(key), 1);
+        }
+
+        /** <inheritDoc /> */
+        public void TryFlush()
+        {
+            ThrowIfDisposed();
+
+            DataStreamerBatch<TK, TV> batch0 = _batch;
+
+            if (batch0 != null)
+                Flush0(batch0, false, PlcFlush);
+        }
+
+        /** <inheritDoc /> */
+        public void Flush()
+        {
+            ThrowIfDisposed();
+
+            DataStreamerBatch<TK, TV> batch0 = _batch;
+
+            if (batch0 != null)
+                Flush0(batch0, true, PlcFlush);
+            else 
+            {
+                // Batch is null, i.e. data streamer is closing. Wait for close to complete.
+                _closedEvt.Wait();
+            }
+        }
+
+        /** <inheritDoc /> */
+        public void Close(bool cancel)
+        {
+            _flusher.Stop();
+
+            while (true)
+            {
+                DataStreamerBatch<TK, TV> batch0 = _batch;
+
+                if (batch0 == null)
+                {
+                    // Wait for concurrent close to finish.
+                    _closedEvt.Wait();
+
+                    return;
+                }
+
+                if (Flush0(batch0, true, cancel ? PlcCancelClose : PlcClose))
+                {
+                    _closeFut.OnDone(null, null);
+
+                    _rwLock.EnterWriteLock(); 
+                    
+                    try
+                    {
+                        base.Dispose(true);
+
+                        if (_rcv != null)
+                            Marshaller.Ignite.HandleRegistry.Release(_rcvHnd);
+
+                        _closedEvt.Set();
+                    }
+                    finally
+                    {
+                        _rwLock.ExitWriteLock();
+                    }
+
+                    Marshaller.Ignite.HandleRegistry.Release(_hnd);
+
+                    break;
+                }
+            }
+        }
+
+        /** <inheritDoc /> */
+        public IDataStreamer<TK1, TV1> WithKeepPortable<TK1, TV1>()
+        {
+            if (_keepPortable)
+            {
+                var result = this as IDataStreamer<TK1, TV1>;
+
+                if (result == null)
+                    throw new InvalidOperationException(
+                        "Can't change type of portable streamer. WithKeepPortable has been called on an instance of " +
+                        "portable streamer with incompatible generic arguments.");
+
+                return result;
+            }
+
+            return new DataStreamerImpl<TK1, TV1>(UU.ProcessorDataStreamer(Marshaller.Ignite.InteropProcessor,
+                _cacheName, true), Marshaller, _cacheName, true);
+        }
+
+        /** <inheritDoc /> */
+        protected override void Dispose(bool disposing)
+        {
+            if (disposing)
+                Close(false);  // Normal dispose: do not cancel
+            else
+            {
+                // Finalizer: just close Java streamer
+                try
+                {
+                    if (_batch != null)
+                        _batch.Send(this, PlcCancelClose);
+                }
+                catch (Exception)
+                {
+                    // Finalizers should never throw
+                }
+
+                Marshaller.Ignite.HandleRegistry.Release(_hnd, true);
+                Marshaller.Ignite.HandleRegistry.Release(_rcvHnd, true);
+
+                base.Dispose(false);
+            }
+        }
+
+        /** <inheritDoc /> */
+        ~DataStreamerImpl()
+        {
+            Dispose(false);
+        }
+
+        /** <inheritDoc /> */
+        public void TopologyChange(long topVer, int topSize)
+        {
+            _rwLock.EnterWriteLock(); 
+            
+            try
+            {
+                ThrowIfDisposed();
+
+                if (_topVer < topVer)
+                {
+                    _topVer = topVer;
+                    _topSize = topSize;
+
+                    _bufSndSize = topSize * UU.DataStreamerPerNodeBufferSizeGet(Target);
+                }
+            }
+            finally
+            {
+                _rwLock.ExitWriteLock();
+            }
+
+        }
+
+        /// <summary>
+        /// Internal add/remove routine.
+        /// </summary>
+        /// <param name="val">Value.</param>
+        /// <param name="cnt">Items count.</param>
+        /// <returns>Future.</returns>
+        private IFuture Add0(object val, int cnt)
+        {
+            int bufSndSize0 = _bufSndSize;
+
+            while (true)
+            {
+                var batch0 = _batch;
+
+                if (batch0 == null)
+                    throw new InvalidOperationException("Data streamer is stopped.");
+
+                int size = batch0.Add(val, cnt);
+
+                if (size == -1)
+                {
+                    // Batch is blocked, perform CAS.
+                    Interlocked.CompareExchange(ref _batch,
+                        new DataStreamerBatch<TK, TV>(batch0), batch0);
+
+                    continue;
+                }
+                if (size >= bufSndSize0)
+                    // Batch is too big, schedule flush.
+                    Flush0(batch0, false, PlcContinue);
+
+                return batch0.Future;
+            }
+        }
+
+        /// <summary>
+        /// Internal flush routine.
+        /// </summary>
+        /// <param name="curBatch"></param>
+        /// <param name="wait">Whether to wait for flush to complete.</param>
+        /// <param name="plc">Whether this is the last batch.</param>
+        /// <returns>Whether this call was able to CAS previous batch</returns>
+        private bool Flush0(DataStreamerBatch<TK, TV> curBatch, bool wait, int plc)
+        {
+            // 1. Try setting new current batch to help further adders. 
+            bool res = Interlocked.CompareExchange(ref _batch, 
+                (plc == PlcContinue || plc == PlcFlush) ? 
+                new DataStreamerBatch<TK, TV>(curBatch) : null, curBatch) == curBatch;
+
+            // 2. Perform actual send.
+            curBatch.Send(this, plc);
+
+            if (wait)
+                // 3. Wait for all futures to finish.
+                curBatch.AwaitCompletion();
+
+            return res;
+        }
+
+        /// <summary>
+        /// Start write.
+        /// </summary>
+        /// <returns>Writer.</returns>
+        internal void Update(Action<PortableWriterImpl> action)
+        {
+            _rwLock.EnterReadLock();
+
+            try
+            {
+                ThrowIfDisposed();
+
+                DoOutOp(OpUpdate, action);
+            }
+            finally
+            {
+                _rwLock.ExitReadLock();
+            }
+        }
+
+        /// <summary>
+        /// Flusher.
+        /// </summary>
+        private class Flusher<TK1, TV1>
+        {
+            /** State: running. */
+            private const int StateRunning = 0;
+
+            /** State: stopping. */
+            private const int StateStopping = 1;
+
+            /** State: stopped. */
+            private const int StateStopped = 2;
+
+            /** Data streamer. */
+            private readonly WeakReference _ldrRef;
+
+            /** Finish flag. */
+            private int _state;
+
+            /** Flush frequency. */
+            private long _freq;
+
+            /// <summary>
+            /// Constructor.
+            /// </summary>
+            /// <param name="ldrRef">Data streamer weak reference..</param>
+            public Flusher(WeakReference ldrRef)
+            {
+                _ldrRef = ldrRef;
+
+                lock (this)
+                {
+                    _state = StateRunning;
+                }
+            }
+
+            /// <summary>
+            /// Main flusher routine.
+            /// </summary>
+            private void Run()
+            {
+                bool force = false;
+                long curFreq = 0;
+                
+                try
+                {
+                    while (true)
+                    {
+                        if (curFreq > 0 || force)
+                        {
+                            var ldr = _ldrRef.Target as DataStreamerImpl<TK1, TV1>;
+
+                            if (ldr == null)
+                                return;
+
+                            ldr.TryFlush();
+
+                            force = false;
+                        }
+
+                        lock (this)
+                        {
+                            // Stop immediately.
+                            if (_state == StateStopping)
+                                return;
+
+                            if (curFreq == _freq)
+                            {
+                                // Frequency is unchanged
+                                if (curFreq == 0)
+                                    // Just wait for a second and re-try.
+                                    Monitor.Wait(this, 1000);
+                                else
+                                {
+                                    // Calculate remaining time.
+                                    DateTime now = DateTime.Now;
+
+                                    long ticks;
+
+                                    try
+                                    {
+                                        ticks = now.AddMilliseconds(curFreq).Ticks - now.Ticks;
+
+                                        if (ticks > int.MaxValue)
+                                            ticks = int.MaxValue;
+                                    }
+                                    catch (ArgumentOutOfRangeException)
+                                    {
+                                        // Handle possible overflow.
+                                        ticks = int.MaxValue;
+                                    }
+
+                                    Monitor.Wait(this, TimeSpan.FromTicks(ticks));
+                                }
+                            }
+                            else
+                            {
+                                if (curFreq != 0)
+                                    force = true;
+
+                                curFreq = _freq;
+                            } 
+                        }
+                    }
+                }
+                finally
+                {
+                    // Let streamer know about stop.
+                    lock (this)
+                    {
+                        _state = StateStopped;
+
+                        Monitor.PulseAll(this);
+                    }
+                }
+            }
+            
+            /// <summary>
+            /// Frequency.
+            /// </summary>
+            public long Frequency
+            {
+                get
+                {
+                    return Interlocked.Read(ref _freq);
+                }
+
+                set
+                {
+                    lock (this)
+                    {
+                        if (_freq != value)
+                        {
+                            _freq = value;
+
+                            Monitor.PulseAll(this);
+                        }
+                    }
+                }
+            }
+
+            /// <summary>
+            /// Stop flusher.
+            /// </summary>
+            public void Stop()
+            {
+                lock (this)
+                {
+                    if (_state == StateRunning)
+                    {
+                        _state = StateStopping;
+
+                        Monitor.PulseAll(this);
+                    }
+
+                    while (_state != StateStopped)
+                        Monitor.Wait(this);
+                }
+            }
+
+            /// <summary>
+            /// Runs the flusher thread.
+            /// </summary>
+            public void RunThread()
+            {
+                new Thread(Run).Start();
+            }
+        }
+
+#pragma warning restore 0420
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerRemoveEntry.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerRemoveEntry.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerRemoveEntry.cs
new file mode 100644
index 0000000..7e65934
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerRemoveEntry.cs
@@ -0,0 +1,48 @@
+/*
+ * 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.Datastream
+{
+    /// <summary>
+    /// Remove marker.
+    /// </summary>
+    internal class DataStreamerRemoveEntry<TK>
+    {
+        /** Key to remove. */
+        private readonly TK _key;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="key">Key.</param>
+        public DataStreamerRemoveEntry(TK key)
+        {
+            _key = key;
+        }
+
+        /// <summary>
+        /// Key.
+        /// </summary>
+        public TK Key
+        {
+            get
+            {
+                return _key;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Datastream/StreamReceiverHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Datastream/StreamReceiverHolder.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Datastream/StreamReceiverHolder.cs
new file mode 100644
index 0000000..5a7c104
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Datastream/StreamReceiverHolder.cs
@@ -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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Datastream
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Datastream;
+    using Apache.Ignite.Core.Impl.Cache;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Portable wrapper for <see cref="IStreamReceiver{TK,TV}"/>.
+    /// </summary>
+    internal class StreamReceiverHolder : IPortableWriteAware
+    {
+        /** */
+        private const byte RcvNormal = 0;
+
+        /** */
+        public const byte RcvTransformer = 1;
+
+        /** Generic receiver. */
+        private readonly object _rcv;
+        
+        /** Invoker delegate. */
+        private readonly Action<object, Ignite, IUnmanagedTarget, IPortableStream, bool> _invoke;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="StreamReceiverHolder"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        public StreamReceiverHolder(PortableReaderImpl reader)
+        {
+            var rcvType = reader.ReadByte();
+
+            _rcv = PortableUtils.ReadPortableOrSerializable<object>(reader);
+            
+            Debug.Assert(_rcv != null);
+
+            var type = _rcv.GetType();
+
+            if (rcvType == RcvTransformer)
+            {
+                // rcv is a user ICacheEntryProcessor<K, V, A, R>, construct StreamTransformer from it.
+                // (we can't marshal StreamTransformer directly, because it is generic, 
+                // and we do not know type arguments that user will have)
+                _rcv = DelegateTypeDescriptor.GetStreamTransformerCtor(type)(_rcv);
+            }
+
+            _invoke = DelegateTypeDescriptor.GetStreamReceiver(_rcv.GetType());
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="StreamReceiverHolder"/> class.
+        /// </summary>
+        /// <param name="rcv">Receiver.</param>
+        /// <param name="invoke">Invoke delegate.</param>
+        public StreamReceiverHolder(object rcv, 
+            Action<object, Ignite, IUnmanagedTarget, IPortableStream, bool> invoke)
+        {
+            Debug.Assert(rcv != null);
+            Debug.Assert(invoke != null);
+
+            _rcv = rcv;
+            _invoke = invoke;
+        }
+
+        /** <inheritdoc /> */
+        public void WritePortable(IPortableWriter writer)
+        {
+            var w = writer.RawWriter();
+
+            var writeAware = _rcv as IPortableWriteAware;
+
+            if (writeAware != null)
+                writeAware.WritePortable(writer);
+            else
+            {
+                w.WriteByte(RcvNormal);
+                PortableUtils.WritePortableOrSerializable((PortableWriterImpl) writer, _rcv);
+            }
+        }
+
+        /// <summary>
+        /// Updates cache with batch of entries.
+        /// </summary>
+        /// <param name="grid">The grid.</param>
+        /// <param name="cache">Cache.</param>
+        /// <param name="stream">Stream.</param>
+        /// <param name="keepPortable">Portable flag.</param>
+        public void Receive(Ignite grid, IUnmanagedTarget cache, IPortableStream stream, bool keepPortable)
+        {
+            Debug.Assert(grid != null);
+            Debug.Assert(cache != null);
+            Debug.Assert(stream != null);
+
+            _invoke(_rcv, grid, cache, stream, keepPortable);
+        }
+
+        /// <summary>
+        /// Invokes the receiver.
+        /// </summary>
+        /// <param name="receiver">Receiver.</param>
+        /// <param name="grid">Grid.</param>
+        /// <param name="cache">Cache.</param>
+        /// <param name="stream">Stream.</param>
+        /// <param name="keepPortable">Portable flag.</param>
+        public static void InvokeReceiver<TK, TV>(IStreamReceiver<TK, TV> receiver, Ignite grid, IUnmanagedTarget cache,
+            IPortableStream stream, bool keepPortable)
+        {
+            var reader = grid.Marshaller.StartUnmarshal(stream, keepPortable);
+
+            var size = reader.ReadInt();
+
+            var entries = new List<ICacheEntry<TK, TV>>(size);
+
+            for (var i = 0; i < size; i++)
+                entries.Add(new CacheEntry<TK, TV>(reader.ReadObject<TK>(), reader.ReadObject<TV>()));
+
+            receiver.Receive(grid.Cache<TK, TV>(cache, keepPortable), entries);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs
new file mode 100644
index 0000000..3972bb0
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs
@@ -0,0 +1,498 @@
+/*
+ * 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.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Linq;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Events;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Handle;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+    using Apache.Ignite.Core.Portable;
+    using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils;
+
+    /// <summary>
+    /// Ignite events.
+    /// </summary>
+    internal class Events : PlatformTarget, IEvents
+    {
+        /// <summary>
+        /// Opcodes.
+        /// </summary>
+        protected enum Op
+        {
+            RemoteQuery = 1,
+            RemoteListen = 2,
+            StopRemoteListen = 3,
+            WaitForLocal = 4,
+            LocalQuery = 5,
+            RecordLocal = 6,
+            EnableLocal = 8,
+            DisableLocal = 9,
+            GetEnabledEvents = 10
+        }
+
+        /** Map from user func to local wrapper, needed for invoke/unsubscribe. */
+        private readonly Dictionary<object, Dictionary<int, LocalHandledEventFilter>> _localFilters
+            = new Dictionary<object, Dictionary<int, LocalHandledEventFilter>>();
+
+        /** Grid. */
+        protected readonly Ignite Ignite;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="Events"/> class.
+        /// </summary>
+        /// <param name="target">Target.</param>
+        /// <param name="marsh">Marshaller.</param>
+        /// <param name="clusterGroup">Cluster group.</param>
+        public Events(IUnmanagedTarget target, PortableMarshaller marsh, IClusterGroup clusterGroup)
+            : base(target, marsh)
+        {
+            Debug.Assert(clusterGroup != null);
+
+            ClusterGroup = clusterGroup;
+
+            Ignite = (Ignite) clusterGroup.Ignite;
+        }
+
+        /** <inheritDoc /> */
+        public virtual IEvents WithAsync()
+        {
+            return new EventsAsync(UU.EventsWithAsync(Target), Marshaller, ClusterGroup);
+        }
+
+        /** <inheritDoc /> */
+        public virtual bool IsAsync
+        {
+            get { return false; }
+        }
+
+        /** <inheritDoc /> */
+        public virtual IFuture GetFuture()
+        {
+            throw IgniteUtils.GetAsyncModeDisabledException();
+        }
+
+        /** <inheritDoc /> */
+        public virtual IFuture<TResult> GetFuture<TResult>()
+        {
+            throw IgniteUtils.GetAsyncModeDisabledException();
+        }
+
+        /** <inheritDoc /> */
+        public IClusterGroup ClusterGroup { get; private set; }
+
+        /** <inheritDoc /> */
+        public virtual List<T> RemoteQuery<T>(IEventFilter<T> filter, TimeSpan? timeout = null, params int[] types)
+            where T : IEvent
+        {
+            IgniteArgumentCheck.NotNull(filter, "filter");
+
+            return DoOutInOp((int) Op.RemoteQuery,
+                writer =>
+                {
+                    writer.Write(new PortableOrSerializableObjectHolder(filter));
+
+                    writer.WriteLong((long) (timeout == null ? 0 : timeout.Value.TotalMilliseconds));
+
+                    WriteEventTypes(types, writer);
+                },
+                reader => ReadEvents<T>(reader));
+        }
+
+        /** <inheritDoc /> */
+        public virtual Guid RemoteListen<T>(int bufSize = 1, TimeSpan? interval = null, bool autoUnsubscribe = true,
+            IEventFilter<T> localListener = null, IEventFilter<T> remoteFilter = null, params int[] types)
+            where T : IEvent
+        {
+            IgniteArgumentCheck.Ensure(bufSize > 0, "bufSize", "should be > 0");
+            IgniteArgumentCheck.Ensure(interval == null || interval.Value.TotalMilliseconds > 0, "interval", "should be null or >= 0");
+
+            return DoOutInOp((int) Op.RemoteListen,
+                writer =>
+                {
+                    writer.WriteInt(bufSize);
+                    writer.WriteLong((long) (interval == null ? 0 : interval.Value.TotalMilliseconds));
+                    writer.WriteBoolean(autoUnsubscribe);
+
+                    writer.WriteBoolean(localListener != null);
+
+                    if (localListener != null)
+                    {
+                        var listener = new RemoteListenEventFilter(Ignite, (id, e) => localListener.Invoke(id, (T) e));
+                        writer.WriteLong(Ignite.HandleRegistry.Allocate(listener));
+                    }
+
+                    writer.WriteBoolean(remoteFilter != null);
+
+                    if (remoteFilter != null)
+                        writer.Write(new PortableOrSerializableObjectHolder(remoteFilter));
+
+                    WriteEventTypes(types, writer);
+                },
+                reader => Marshaller.StartUnmarshal(reader).ReadGuid() ?? Guid.Empty);
+        }
+
+        /** <inheritDoc /> */
+        public virtual void StopRemoteListen(Guid opId)
+        {
+            DoOutOp((int) Op.StopRemoteListen, writer =>
+            {
+                Marshaller.StartMarshal(writer).WriteGuid(opId);
+            });
+        }
+
+        /** <inheritDoc /> */
+        public IEvent WaitForLocal(params int[] types)
+        {
+            return WaitForLocal<IEvent>(null, types);
+        }
+
+        /** <inheritDoc /> */
+        public virtual T WaitForLocal<T>(IEventFilter<T> filter, params int[] types) where T : IEvent
+        {
+            long hnd = 0;
+
+            try
+            {
+                return WaitForLocal0(filter, ref hnd, types);
+            }
+            finally
+            {
+                if (filter != null)
+                    Ignite.HandleRegistry.Release(hnd);
+            }
+        }
+
+        /** <inheritDoc /> */
+        public List<IEvent> LocalQuery(params int[] types)
+        {
+            return DoOutInOp((int) Op.LocalQuery,
+                writer => WriteEventTypes(types, writer),
+                reader => ReadEvents<IEvent>(reader));
+        }
+
+        /** <inheritDoc /> */
+        public void RecordLocal(IEvent evt)
+        {
+            throw new NotImplementedException("GG-10244");
+        }
+
+        /** <inheritDoc /> */
+        public void LocalListen<T>(IEventFilter<T> listener, params int[] types) where T : IEvent
+        {
+            IgniteArgumentCheck.NotNull(listener, "listener");
+            IgniteArgumentCheck.NotNullOrEmpty(types, "types");
+
+            foreach (var type in types)
+                LocalListen(listener, type);
+        }
+
+        /** <inheritDoc /> */
+        public bool StopLocalListen<T>(IEventFilter<T> listener, params int[] types) where T : IEvent
+        {
+            lock (_localFilters)
+            {
+                Dictionary<int, LocalHandledEventFilter> filters;
+
+                if (!_localFilters.TryGetValue(listener, out filters))
+                    return false;
+
+                var success = false;
+
+                // Should do this inside lock to avoid race with subscription
+                // ToArray is required because we are going to modify underlying dictionary during enumeration
+                foreach (var filter in GetLocalFilters(listener, types).ToArray())
+                    success |= UU.EventsStopLocalListen(Target, filter.Handle);
+
+                return success;
+            }
+        }
+
+        /** <inheritDoc /> */
+        public void EnableLocal(params int[] types)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(types, "types");
+
+            DoOutOp((int)Op.EnableLocal, writer => WriteEventTypes(types, writer));
+        }
+
+        /** <inheritDoc /> */
+        public void DisableLocal(params int[] types)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(types, "types");
+
+            DoOutOp((int)Op.DisableLocal, writer => WriteEventTypes(types, writer));
+        }
+
+        /** <inheritDoc /> */
+        public int[] GetEnabledEvents()
+        {
+            return DoInOp((int)Op.GetEnabledEvents, reader => ReadEventTypes(reader));
+        }
+
+        /** <inheritDoc /> */
+        public bool IsEnabled(int type)
+        {
+            return UU.EventsIsEnabled(Target, type);
+        }
+
+        /// <summary>
+        /// Waits for the specified events.
+        /// </summary>
+        /// <typeparam name="T">Type of events.</typeparam>
+        /// <param name="filter">Optional filtering predicate. Event wait will end as soon as it returns false.</param>
+        /// <param name="handle">The filter handle, if applicable.</param>
+        /// <param name="types">Types of the events to wait for. 
+        /// If not provided, all events will be passed to the filter.</param>
+        /// <returns>Ignite event.</returns>
+        protected T WaitForLocal0<T>(IEventFilter<T> filter, ref long handle, params int[] types) where T : IEvent
+        {
+            if (filter != null)
+                handle = Ignite.HandleRegistry.Allocate(new LocalEventFilter
+                {
+                    InvokeFunc = stream => InvokeLocalFilter(stream, filter)
+                });
+
+            var hnd = handle;
+
+            return DoOutInOp((int)Op.WaitForLocal,
+                writer =>
+                {
+                    if (filter != null)
+                    {
+                        writer.WriteBoolean(true);
+                        writer.WriteLong(hnd);
+                    }
+                    else
+                        writer.WriteBoolean(false);
+
+                    WriteEventTypes(types, writer);
+                },
+                reader => EventReader.Read<T>(Marshaller.StartUnmarshal(reader)));
+        }
+
+        /// <summary>
+        /// Reads events from a portable stream.
+        /// </summary>
+        /// <typeparam name="T">Event type.</typeparam>
+        /// <param name="reader">Reader.</param>
+        /// <returns>Resulting list or null.</returns>
+        private List<T> ReadEvents<T>(IPortableStream reader) where T : IEvent
+        {
+            return ReadEvents<T>(Marshaller.StartUnmarshal(reader));
+        }
+
+        /// <summary>
+        /// Reads events from a portable reader.
+        /// </summary>
+        /// <typeparam name="T">Event type.</typeparam>
+        /// <param name="portableReader">Reader.</param>
+        /// <returns>Resulting list or null.</returns>
+        protected static List<T> ReadEvents<T>(PortableReaderImpl portableReader) where T : IEvent
+        {
+            var count = portableReader.RawReader().ReadInt();
+
+            if (count == -1)
+                return null;
+
+            var result = new List<T>(count);
+
+            for (var i = 0; i < count; i++)
+                result.Add(EventReader.Read<T>(portableReader));
+
+            return result;
+        }
+
+        /// <summary>
+        /// Gets local filters by user listener and event type.
+        /// </summary>
+        /// <param name="listener">Listener.</param>
+        /// <param name="types">Types.</param>
+        /// <returns>Collection of local listener wrappers.</returns>
+        [SuppressMessage("ReSharper", "InconsistentlySynchronizedField",
+            Justification = "This private method should be always called within a lock on localFilters")]
+        private IEnumerable<LocalHandledEventFilter> GetLocalFilters(object listener, int[] types)
+        {
+            Dictionary<int, LocalHandledEventFilter> filters;
+
+            if (!_localFilters.TryGetValue(listener, out filters))
+                return Enumerable.Empty<LocalHandledEventFilter>();
+
+            if (types.Length == 0)
+                return filters.Values;
+
+            return types.Select(type =>
+            {
+                LocalHandledEventFilter filter;
+
+                return filters.TryGetValue(type, out filter) ? filter : null;
+            }).Where(x => x != null);
+        }
+
+        /// <summary>
+        /// Adds an event listener for local events.
+        /// </summary>
+        /// <typeparam name="T">Type of events.</typeparam>
+        /// <param name="listener">Predicate that is called on each received event.</param>
+        /// <param name="type">Event type for which this listener will be notified</param>
+        private void LocalListen<T>(IEventFilter<T> listener, int type) where T : IEvent
+        {
+            lock (_localFilters)
+            {
+                Dictionary<int, LocalHandledEventFilter> filters;
+
+                if (!_localFilters.TryGetValue(listener, out filters))
+                {
+                    filters = new Dictionary<int, LocalHandledEventFilter>();
+
+                    _localFilters[listener] = filters;
+                }
+
+                LocalHandledEventFilter localFilter;
+
+                if (!filters.TryGetValue(type, out localFilter))
+                {
+                    localFilter = CreateLocalFilter(listener, type);
+
+                    filters[type] = localFilter;
+                }
+
+                UU.EventsLocalListen(Target, localFilter.Handle, type);
+            }
+        }
+
+        /// <summary>
+        /// Creates a user filter wrapper.
+        /// </summary>
+        /// <typeparam name="T">Event object type.</typeparam>
+        /// <param name="listener">Listener.</param>
+        /// <param name="type">Event type.</param>
+        /// <returns>Created wrapper.</returns>
+        private LocalHandledEventFilter CreateLocalFilter<T>(IEventFilter<T> listener, int type) where T : IEvent
+        {
+            var result = new LocalHandledEventFilter(
+                stream => InvokeLocalFilter(stream, listener),
+                unused =>
+                {
+                    lock (_localFilters)
+                    {
+                        Dictionary<int, LocalHandledEventFilter> filters;
+
+                        if (_localFilters.TryGetValue(listener, out filters))
+                        {
+                            filters.Remove(type);
+
+                            if (filters.Count == 0)
+                                _localFilters.Remove(listener);
+                        }
+                    }
+                });
+
+            result.Handle = Ignite.HandleRegistry.Allocate(result);
+
+            return result;
+        }
+
+        /// <summary>
+        /// Invokes local filter using data from specified stream.
+        /// </summary>
+        /// <typeparam name="T">Event object type.</typeparam>
+        /// <param name="stream">The stream.</param>
+        /// <param name="listener">The listener.</param>
+        /// <returns>Filter invocation result.</returns>
+        private bool InvokeLocalFilter<T>(IPortableStream stream, IEventFilter<T> listener) where T : IEvent
+        {
+            var evt = EventReader.Read<T>(Marshaller.StartUnmarshal(stream));
+
+            // No guid in local mode
+            return listener.Invoke(Guid.Empty, evt);
+        }
+
+        /// <summary>
+        /// Writes the event types.
+        /// </summary>
+        /// <param name="types">Types.</param>
+        /// <param name="writer">Writer.</param>
+        private static void WriteEventTypes(int[] types, IPortableRawWriter writer)
+        {
+            if (types.Length == 0)
+                types = null;  // empty array means no type filtering
+
+            writer.WriteIntArray(types);
+        }
+
+        /// <summary>
+        /// Writes the event types.
+        /// </summary>
+        /// <param name="reader">Reader.</param>
+        private int[] ReadEventTypes(IPortableStream reader)
+        {
+            return Marshaller.StartUnmarshal(reader).ReadIntArray();
+        }
+
+        /// <summary>
+        /// Local user filter wrapper.
+        /// </summary>
+        private class LocalEventFilter : IInteropCallback
+        {
+            /** */
+            public Func<IPortableStream, bool> InvokeFunc;
+
+            /** <inheritdoc /> */
+            public int Invoke(IPortableStream stream)
+            {
+                return InvokeFunc(stream) ? 1 : 0;
+            }
+        }
+
+        /// <summary>
+        /// Local user filter wrapper with handle.
+        /// </summary>
+        private class LocalHandledEventFilter : Handle<Func<IPortableStream, bool>>, IInteropCallback
+        {
+            /** */
+            public long Handle;
+
+            /** <inheritdoc /> */
+            public int Invoke(IPortableStream stream)
+            {
+                return Target(stream) ? 1 : 0;
+            }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="LocalHandledEventFilter"/> class.
+            /// </summary>
+            /// <param name="invokeFunc">The invoke function.</param>
+            /// <param name="releaseAction">The release action.</param>
+            public LocalHandledEventFilter(
+                Func<IPortableStream, bool> invokeFunc, Action<Func<IPortableStream, bool>> releaseAction) 
+                : base(invokeFunc, releaseAction)
+            {
+                // No-op.
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Events/EventsAsync.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Events/EventsAsync.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Events/EventsAsync.cs
new file mode 100644
index 0000000..632d8b8
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Events/EventsAsync.cs
@@ -0,0 +1,158 @@
+/*
+ * 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.Diagnostics.CodeAnalysis;
+    using System.Threading;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Events;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+    using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils;
+
+    /// <summary>
+    /// Async Ignite events.
+    /// </summary>
+    [SuppressMessage("Microsoft.Design", "CA1001:TypesThatOwnDisposableFieldsShouldBeDisposable")]
+    internal class EventsAsync : Events
+    {
+        /** */
+        private readonly ThreadLocal<int> _lastAsyncOp = new ThreadLocal<int>(() => OpNone);
+
+        /** */
+        private readonly ThreadLocal<IFuture> _curFut = new ThreadLocal<IFuture>();
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="Events"/> class.
+        /// </summary>
+        /// <param name="target">Target.</param>
+        /// <param name="marsh">Marshaller.</param>
+        /// <param name="clusterGroup">Cluster group.</param>
+        public EventsAsync(IUnmanagedTarget target, PortableMarshaller marsh, IClusterGroup clusterGroup)
+            : base(target, marsh, clusterGroup)
+        {
+            // No-op.
+        }
+
+        /** <inheritdoc /> */
+        public override List<T> RemoteQuery<T>(IEventFilter<T> filter, TimeSpan? timeout = null, params int[] types)
+        {
+            _lastAsyncOp.Value = (int) Op.RemoteQuery;
+
+            var result = base.RemoteQuery(filter, timeout, types);
+
+            // Result is a List<T> so we can't create proper converter later in GetFuture call from user.
+            // ReSharper disable once RedundantTypeArgumentsOfMethod (otherwise won't compile in VS2010 / TC)
+            _curFut.Value = GetFuture<List<T>>((futId, futTyp) => UU.TargetListenFutureForOperation(Target, futId, futTyp,
+                (int) Op.RemoteQuery), convertFunc: ReadEvents<T>);
+
+            return result;
+        }
+
+        /** <inheritdoc /> */
+        public override Guid RemoteListen<T>(int bufSize = 1, TimeSpan? interval = null, bool autoUnsubscribe = true,
+            IEventFilter<T> localListener = null, IEventFilter<T> remoteFilter = null, params int[] types)
+        {
+            _lastAsyncOp.Value = (int) Op.RemoteListen;
+            _curFut.Value = null;
+
+            return base.RemoteListen(bufSize, interval, autoUnsubscribe, localListener, remoteFilter, types);
+        }
+
+        /** <inheritdoc /> */
+        public override void StopRemoteListen(Guid opId)
+        {
+            _lastAsyncOp.Value = (int) Op.StopRemoteListen;
+            _curFut.Value = null;
+
+            base.StopRemoteListen(opId);
+        }
+
+        /** <inheritdoc /> */
+        public override T WaitForLocal<T>(IEventFilter<T> filter, params int[] types)
+        {
+            _lastAsyncOp.Value = (int) Op.WaitForLocal;
+
+            long hnd = 0;
+
+            try
+            {
+                var result = WaitForLocal0(filter, ref hnd, types);
+
+                if (filter != null)
+                {
+                    // Dispose handle as soon as future ends.
+                    var fut = GetFuture<T>();
+
+                    _curFut.Value = fut;
+
+                    fut.Listen(() => Ignite.HandleRegistry.Release(hnd));
+                }
+                else
+                    _curFut.Value = null;
+
+                return result;
+            }
+            catch (Exception)
+            {
+                Ignite.HandleRegistry.Release(hnd);
+                throw;
+            }
+        }
+
+        /** <inheritdoc /> */
+        public override IEvents WithAsync()
+        {
+            return this;
+        }
+
+        /** <inheritdoc /> */
+        public override bool IsAsync
+        {
+            get { return true; }
+        }
+
+        /** <inheritdoc /> */
+        public override IFuture GetFuture()
+        {
+            return GetFuture<object>();
+        }
+
+        /** <inheritdoc /> */
+        public override IFuture<T> GetFuture<T>()
+        {
+            if (_curFut.Value != null)
+            {
+                var fut = _curFut.Value;
+                _curFut.Value = null;
+                return (IFuture<T>) fut;
+            }
+
+            Func<PortableReaderImpl, T> converter = null;
+
+            if (_lastAsyncOp.Value == (int) Op.WaitForLocal)
+                converter = reader => (T) EventReader.Read<IEvent>(reader);
+
+            return GetFuture((futId, futTyp) => UU.TargetListenFutureForOperation(Target, futId, futTyp, _lastAsyncOp.Value),
+                convertFunc: converter);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Events/RemoteListenEventFilter.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Events/RemoteListenEventFilter.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Events/RemoteListenEventFilter.cs
new file mode 100644
index 0000000..8b44966
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Events/RemoteListenEventFilter.cs
@@ -0,0 +1,85 @@
+/*
+ * 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.Diagnostics;
+    using Apache.Ignite.Core.Events;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+
+    /// <summary>
+    /// Event filter/listener holder for RemoteListen.
+    /// </summary>
+    internal class RemoteListenEventFilter : IInteropCallback
+    {
+        /** */
+        private readonly Ignite _ignite;
+        
+        /** */
+        private readonly Func<Guid, IEvent, bool> _filter;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="RemoteListenEventFilter"/> class.
+        /// </summary>
+        /// <param name="ignite">The grid.</param>
+        /// <param name="filter">The filter.</param>
+        public RemoteListenEventFilter(Ignite ignite, Func<Guid, IEvent, bool> filter)
+        {
+            _ignite = ignite;
+            _filter = filter;
+        }
+
+        /** <inheritdoc /> */
+        public int Invoke(IPortableStream stream)
+        {
+            var reader = _ignite.Marshaller.StartUnmarshal(stream);
+
+            var evt = EventReader.Read<IEvent>(reader);
+
+            var nodeId = reader.ReadGuid() ?? Guid.Empty;
+
+            return _filter(nodeId, evt) ? 1 : 0;
+        }
+
+        /// <summary>
+        /// Creates an instance of this class from a stream.
+        /// </summary>
+        /// <param name="memPtr">Memory pointer.</param>
+        /// <param name="grid">Grid</param>
+        /// <returns>Deserialized instance of <see cref="RemoteListenEventFilter"/></returns>
+        public static RemoteListenEventFilter CreateInstance(long memPtr, Ignite grid)
+        {
+            Debug.Assert(grid != null);
+
+            using (var stream = IgniteManager.Memory.Get(memPtr).Stream())
+            {
+                var marsh = grid.Marshaller;
+
+                var reader = marsh.StartUnmarshal(stream);
+
+                var pred = reader.ReadObject<PortableOrSerializableObjectHolder>().Item;
+
+                var func = DelegateTypeDescriptor.GetEventFilter(pred.GetType());
+
+                return new RemoteListenEventFilter(grid, (id, evt) => func(pred, id, evt));
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
new file mode 100644
index 0000000..066f345
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
@@ -0,0 +1,204 @@
+/*
+ * 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
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using System.Runtime.InteropServices;
+    using System.Security;
+    using System.Threading;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cache.Store;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Transactions;
+
+    /// <summary>
+    /// Managed environment. Acts as a gateway for native code.
+    /// </summary>
+    [StructLayout(LayoutKind.Sequential)]
+    internal static class ExceptionUtils
+    {
+        /** NoClassDefFoundError fully-qualified class name which is important during startup phase. */
+        private const string ClsNoClsDefFoundErr = "java.lang.NoClassDefFoundError";
+
+        /** NoSuchMethodError fully-qualified class name which is important during startup phase. */
+        private const string ClsNoSuchMthdErr = "java.lang.NoSuchMethodError";
+
+        /** InteropCachePartialUpdateException. */
+        private const string ClsCachePartialUpdateErr = "org.apache.ignite.internal.processors.platform.cache.PlatformCachePartialUpdateException";
+        
+        /** Map with predefined exceptions. */
+        private static readonly IDictionary<string, ExceptionFactoryDelegate> EXS = new Dictionary<string, ExceptionFactoryDelegate>();
+
+        /** Exception factory delegate. */
+        private delegate Exception ExceptionFactoryDelegate(string msg);
+        
+        /// <summary>
+        /// Static initializer.
+        /// </summary>
+        static ExceptionUtils()
+        {
+            // Common Java exceptions mapped to common .Net exceptions.
+            EXS["java.lang.IllegalArgumentException"] = m => new ArgumentException(m);
+            EXS["java.lang.IllegalStateException"] = m => new InvalidOperationException(m);
+            EXS["java.lang.UnsupportedOperationException"] = m => new NotImplementedException(m);
+            EXS["java.lang.InterruptedException"] = m => new ThreadInterruptedException(m);
+            
+            // Generic Ignite exceptions.
+            EXS["org.apache.ignite.IgniteException"] = m => new IgniteException(m);
+            EXS["org.apache.ignite.IgniteCheckedException"] = m => new IgniteException(m);
+
+            // Cluster exceptions.
+            EXS["org.apache.ignite.cluster.ClusterGroupEmptyException"] = m => new ClusterGroupEmptyException(m);
+            EXS["org.apache.ignite.cluster.ClusterTopologyException"] = m => new ClusterTopologyException(m);
+
+            // Compute exceptions.
+            EXS["org.apache.ignite.compute.ComputeExecutionRejectedException"] = m => new ComputeExecutionRejectedException(m);
+            EXS["org.apache.ignite.compute.ComputeJobFailoverException"] = m => new ComputeJobFailoverException(m);
+            EXS["org.apache.ignite.compute.ComputeTaskCancelledException"] = m => new ComputeTaskCancelledException(m);
+            EXS["org.apache.ignite.compute.ComputeTaskTimeoutException"] = m => new ComputeTaskTimeoutException(m);
+            EXS["org.apache.ignite.compute.ComputeUserUndeclaredException"] = m => new ComputeUserUndeclaredException(m);
+
+            // Cache exceptions.
+            EXS["javax.cache.CacheException"] = m => new CacheException(m);
+            EXS["javax.cache.integration.CacheLoaderException"] = m => new CacheStoreException(m);
+            EXS["javax.cache.integration.CacheWriterException"] = m => new CacheStoreException(m);
+            EXS["javax.cache.processor.EntryProcessorException"] = m => new CacheEntryProcessorException(m);
+            EXS["org.apache.ignite.cache.CacheAtomicUpdateTimeoutException"] = m => new CacheAtomicUpdateTimeoutException(m);
+            
+            // Transaction exceptions.
+            EXS["org.apache.ignite.transactions.TransactionOptimisticException"] = m => new TransactionOptimisticException(m);
+            EXS["org.apache.ignite.transactions.TransactionTimeoutException"] = m => new TransactionTimeoutException(m);
+            EXS["org.apache.ignite.transactions.TransactionRollbackException"] = m => new TransactionRollbackException(m);
+            EXS["org.apache.ignite.transactions.TransactionHeuristicException"] = m => new TransactionHeuristicException(m);
+
+            // Security exceptions.
+            EXS["org.apache.ignite.IgniteAuthenticationException"] = m => new SecurityException(m);
+            EXS["org.apache.ignite.plugin.security.GridSecurityException"] = m => new SecurityException(m);
+        }
+
+        /// <summary>
+        /// Creates exception according to native code class and message.
+        /// </summary>
+        /// <param name="clsName">Exception class name.</param>
+        /// <param name="msg">Exception message.</param>
+        /// <param name="reader">Error data reader.</param>
+        public static Exception GetException(string clsName, string msg, PortableReaderImpl reader = null)
+        {
+            ExceptionFactoryDelegate ctor;
+
+            if (EXS.TryGetValue(clsName, out ctor))
+                return ctor(msg);
+
+            if (ClsNoClsDefFoundErr.Equals(clsName))
+                return new IgniteException("Java class is not found (did you set IGNITE_HOME environment " +
+                    "variable?): " + msg);
+
+            if (ClsNoSuchMthdErr.Equals(clsName))
+                return new IgniteException("Java class method is not found (did you set IGNITE_HOME environment " +
+                    "variable?): " + msg);
+
+            if (ClsCachePartialUpdateErr.Equals(clsName))
+                return ProcessCachePartialUpdateException(msg, reader);
+            
+            return new IgniteException("Java exception occurred [class=" + clsName + ", message=" + msg + ']');
+        }
+
+        /// <summary>
+        /// Process cache partial update exception.
+        /// </summary>
+        /// <param name="msg">Message.</param>
+        /// <param name="reader">Reader.</param>
+        /// <returns></returns>
+        private static Exception ProcessCachePartialUpdateException(string msg, PortableReaderImpl reader)
+        {
+            if (reader == null)
+                return new CachePartialUpdateException(msg, new IgniteException("Failed keys are not available."));
+            
+            bool dataExists = reader.ReadBoolean();
+
+            Debug.Assert(dataExists);
+
+            if (reader.ReadBoolean())
+            {
+                bool keepPortable = reader.ReadBoolean();
+
+                PortableReaderImpl keysReader = reader.Marshaller.StartUnmarshal(reader.Stream, keepPortable);
+
+                try
+                {
+                    return new CachePartialUpdateException(msg, ReadNullableList(keysReader));
+                }
+                catch (Exception e)
+                {
+                    // Failed to deserialize data.
+                    return new CachePartialUpdateException(msg, e);
+                }
+            }
+            
+            // Was not able to write keys.
+            string innerErrCls = reader.ReadString();
+            string innerErrMsg = reader.ReadString();
+
+            Exception innerErr = GetException(innerErrCls, innerErrMsg);
+
+            return new CachePartialUpdateException(msg, innerErr);
+        }
+
+        /// <summary>
+        /// Create JVM initialization exception.
+        /// </summary>
+        /// <param name="clsName">Class name.</param>
+        /// <param name="msg">Message.</param>
+        /// <returns>Exception.</returns>
+        public static Exception GetJvmInitializeException(string clsName, string msg)
+        {
+            if (clsName != null)
+                return new IgniteException("Failed to initialize JVM.", GetException(clsName, msg));
+
+            if (msg != null)
+                return new IgniteException("Failed to initialize JVM: " + msg);
+
+            return new IgniteException("Failed to initialize JVM.");
+        }
+
+        /// <summary>
+        /// Reads nullable list.
+        /// </summary>
+        /// <param name="reader">Reader.</param>
+        /// <returns>List.</returns>
+        private static List<object> ReadNullableList(PortableReaderImpl reader)
+        {
+            if (!reader.ReadBoolean()) 
+                return null;
+
+            var size = reader.ReadInt();
+
+            var list = new List<object>(size);
+
+            for (int i = 0; i < size; i++)
+                list.Add(reader.ReadObject<object>());
+
+            return list;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Handle/HandleRegistry.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Handle/HandleRegistry.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Handle/HandleRegistry.cs
index 2a67c41..9c8178f 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Handle/HandleRegistry.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Handle/HandleRegistry.cs
@@ -253,7 +253,7 @@ namespace Apache.Ignite.Core.Impl.Handle
             }
 
             if (throwOnAbsent)
-                throw new InvalidOperationException("Resource handle has been released (is grid stopping?).");
+                throw new InvalidOperationException("Resource handle has been released (is Ignite stopping?).");
 
             return default(T);
         }
@@ -333,7 +333,7 @@ namespace Apache.Ignite.Core.Impl.Handle
         /// <returns>Exception.</returns>
         private static Exception ClosedException()
         {
-            return new InvalidOperationException("Cannot allocate a resource handle because grid is stopping.");
+            return new InvalidOperationException("Cannot allocate a resource handle because Ignite is stopping.");
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/IInteropCallback.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/IInteropCallback.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/IInteropCallback.cs
new file mode 100644
index 0000000..91838d0
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/IInteropCallback.cs
@@ -0,0 +1,34 @@
+/*
+ * 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
+{
+    using Apache.Ignite.Core.Impl.Portable.IO;
+
+    /// <summary>
+    /// Interop callback.
+    /// </summary>
+    internal interface IInteropCallback
+    {
+        /// <summary>
+        /// Invokes callback.
+        /// </summary>
+        /// <param name="stream">Stream.</param>
+        /// <returns>Invocation result.</returns>
+        int Invoke(IPortableStream stream);
+    }
+}
\ No newline at end of file


[11/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/AbstractTaskTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/AbstractTaskTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/AbstractTaskTest.cs
new file mode 100644
index 0000000..f370740
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/AbstractTaskTest.cs
@@ -0,0 +1,217 @@
+/*
+ * 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.Compute
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Threading;
+    using Apache.Ignite.Core.Portable;
+    using Apache.Ignite.Core.Tests.Process;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Base class for all task-related tests.
+    /// </summary>
+    public abstract class AbstractTaskTest
+    {
+        /** */
+        protected const string Grid1Name = "grid1";
+
+        /** */
+        protected const string Grid2Name = "grid2";
+
+        /** */
+        protected const string Grid3Name = "grid3";
+
+        /** */
+        protected const string Cache1Name = "cache1";
+
+        /** Whether this is a test with forked JVMs. */
+        private readonly bool _fork;
+
+        /** First node. */
+        [NonSerialized]
+        protected IIgnite Grid1;
+
+        /** Second node. */
+        [NonSerialized]
+        private IIgnite _grid2;
+
+        /** Third node. */
+        [NonSerialized]
+        private IIgnite _grid3;
+
+        /** Second process. */
+        [NonSerialized]
+        private IgniteProcess _proc2;
+
+        /** Third process. */
+        [NonSerialized]
+        private IgniteProcess _proc3;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="fork">Fork flag.</param>
+        protected AbstractTaskTest(bool fork)
+        {
+            _fork = fork;
+        }
+
+        /// <summary>
+        /// Initialization routine.
+        /// </summary>
+        [TestFixtureSetUp]
+        public void InitClient()
+        {
+            TestUtils.KillProcesses();
+
+            if (_fork)
+            {
+                Grid1 = Ignition.Start(Configuration("config\\compute\\compute-standalone.xml"));
+
+                _proc2 = Fork("config\\compute\\compute-standalone.xml");
+
+                while (true)
+                {
+                    if (!_proc2.Alive)
+                        throw new Exception("Process 2 died unexpectedly: " + _proc2.Join());
+
+                    if (Grid1.Cluster.Nodes().Count < 2)
+                        Thread.Sleep(100);
+                    else
+                        break;
+                }
+
+                _proc3 = Fork("config\\compute\\compute-standalone.xml");
+
+                while (true)
+                {
+                    if (!_proc3.Alive)
+                        throw new Exception("Process 3 died unexpectedly: " + _proc3.Join());
+
+                    if (Grid1.Cluster.Nodes().Count < 3)
+                        Thread.Sleep(100);
+                    else
+                        break;
+                }
+            }
+            else
+            {
+                Grid1 = Ignition.Start(Configuration("config\\compute\\compute-grid1.xml"));
+                _grid2 = Ignition.Start(Configuration("config\\compute\\compute-grid2.xml"));
+                _grid3 = Ignition.Start(Configuration("config\\compute\\compute-grid3.xml"));
+            }
+        }
+
+        [SetUp]
+        public void BeforeTest()
+        {
+            Console.WriteLine("Test started: " + TestContext.CurrentContext.Test.Name);
+        }
+
+        [TestFixtureTearDown]
+        public void StopClient()
+        {
+            if (Grid1 != null)
+                Ignition.Stop(Grid1.Name, true);
+
+            if (_fork)
+            {
+                if (_proc2 != null) {
+                    _proc2.Kill();
+
+                    _proc2.Join();
+                }
+
+                if (_proc3 != null)
+                {
+                    _proc3.Kill();
+
+                    _proc3.Join();
+                }
+            }
+            else
+            {
+                if (_grid2 != null)
+                    Ignition.Stop(_grid2.Name, true);
+
+                if (_grid3 != null)
+                    Ignition.Stop(_grid3.Name, true);
+            }
+        }
+
+        /// <summary>
+        /// Configuration for node.
+        /// </summary>
+        /// <param name="path">Path to Java XML configuration.</param>
+        /// <returns>Node configuration.</returns>
+        protected IgniteConfiguration Configuration(string path)
+        {
+            IgniteConfiguration cfg = new IgniteConfiguration();
+
+            if (!_fork)
+            {
+                PortableConfiguration portCfg = new PortableConfiguration();
+
+                ICollection<PortableTypeConfiguration> portTypeCfgs = new List<PortableTypeConfiguration>();
+
+                PortableTypeConfigurations(portTypeCfgs);
+
+                portCfg.TypeConfigurations = portTypeCfgs;
+
+                cfg.PortableConfiguration = portCfg;
+            }
+
+            cfg.JvmClasspath = TestUtils.CreateTestClasspath();
+
+            cfg.JvmOptions = TestUtils.TestJavaOptions();
+
+            cfg.SpringConfigUrl = path;
+
+            return cfg;
+        }
+
+        /// <summary>
+        /// Create forked process with the following Spring config.
+        /// </summary>
+        /// <param name="path">Path to Java XML configuration.</param>
+        /// <returns>Forked process.</returns>
+        private static IgniteProcess Fork(string path)
+        {
+            return new IgniteProcess(
+                "-springConfigUrl=" + path,
+                "-J-ea",
+                "-J-Xcheck:jni",
+                "-J-Xms512m",
+                "-J-Xmx512m",
+                "-J-DIGNITE_QUIET=false"
+                //"-J-Xnoagent", "-J-Djava.compiler=NONE", "-J-Xdebug", "-J-Xrunjdwp:transport=dt_socket,server=y,suspend=n,address=5006"
+            );
+        }
+
+        /// <summary>
+        /// Define portable types.
+        /// </summary>
+        /// <param name="portTypeCfgs">Portable type configurations.</param>
+        protected virtual void PortableTypeConfigurations(ICollection<PortableTypeConfiguration> portTypeCfgs)
+        {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/ClosureTaskTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/ClosureTaskTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/ClosureTaskTest.cs
new file mode 100644
index 0000000..047e46b
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/ClosureTaskTest.cs
@@ -0,0 +1,390 @@
+/*
+ * 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.Compute
+{
+    using System;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Compute;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests for distributed closure executions.
+    /// </summary>
+    public abstract class ClosureTaskTest : AbstractTaskTest
+    {
+        /** Amount of multiple clousres. */
+        private const int MultiCloCnt = 5;
+
+        /** */
+        protected const string ErrMsg = "An error has occurred.";
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="fork">Fork mode.</param>
+        protected ClosureTaskTest(bool fork) : base(fork) { }
+
+        /// <summary>
+        /// Test for single closure returning result.
+        /// </summary>
+        [Test]
+        public void TestExecuteSingle()
+        {
+            var res = Grid1.Compute().Call(OutFunc(false));
+
+            CheckResult(res);
+        }
+
+        /// <summary>
+        /// Test for single closure returning exception.
+        /// </summary>
+        [Test]
+        public void TestExecuteSingleException()
+        {
+            try
+            {
+                Grid1.Compute().Call(OutFunc(true));
+
+                Assert.Fail();
+            }
+            catch (Exception e)
+            {
+                CheckError(e);
+            }
+        }
+
+        /// <summary>
+        /// Test for multiple closures execution.
+        /// </summary>
+        [Test]
+        public void TestExecuteMultiple()
+        {
+            var clos = new List<IComputeFunc<object>>(MultiCloCnt);
+
+            for (int i = 0; i < MultiCloCnt; i++)
+                clos.Add(OutFunc(false));
+
+            ICollection<object> ress = Grid1.Compute().Call(clos);
+
+            foreach (object res in ress)
+                CheckResult(res);
+        }
+
+        /// <summary>
+        /// Test for multiple closures execution.
+        /// </summary>
+        [Test]
+        public void TestExecuteMultipleReduced()
+        {
+            var clos = new List<IComputeFunc<object>>(MultiCloCnt);
+
+            for (int i = 0; i < MultiCloCnt; i++)
+                clos.Add(OutFunc(false));
+
+            ICollection<object> ress = Grid1.Compute().Call(clos, new Reducer(false));
+
+            foreach (object res in ress)
+                CheckResult(res);
+        }
+
+        /// <summary>
+        /// Test for multiple closures execution with exceptions thrown from some of them.
+        /// </summary>
+        [Test]
+        public void TestExecuteMultipleException()
+        {
+            var clos = new List<IComputeFunc<object>>(MultiCloCnt);
+
+            for (int i = 0; i < MultiCloCnt; i++)
+                clos.Add(OutFunc(i % 2 == 0)); // Some closures will be faulty.
+
+            try
+            {
+                Grid1.Compute().Call(clos);
+
+                Assert.Fail();
+            }
+            catch (Exception e)
+            {
+                CheckError(e);
+            }
+        }
+
+        /// <summary>
+        /// Test broadcast out-closure execution.
+        /// </summary>
+        [Test]
+        public void TestBroadcastOut()
+        {
+            ICollection<object> ress = Grid1.Compute().Broadcast(OutFunc(false));
+
+            foreach (object res in ress)
+                CheckResult(res);
+        }
+
+        /// <summary>
+        /// Test broadcast out-closure execution with exception.
+        /// </summary>
+        [Test]
+        public void TestBroadcastOutException()
+        {
+            try
+            {
+                Grid1.Compute().Broadcast(OutFunc(true));
+
+                Assert.Fail();
+            }
+            catch (Exception e)
+            {
+                CheckError(e);
+            }
+        }
+
+        /// <summary>
+        /// Test broadcast in-out-closure execution.
+        /// </summary>
+        [Test]
+        public void TestBroadcastInOut()
+        {
+            ICollection<object> ress = Grid1.Compute().Broadcast(Func(false), 1);
+
+            foreach (object res in ress)
+                CheckResult(res);
+        }
+
+        /// <summary>
+        /// Test broadcast in-out-closure execution with exception.
+        /// </summary>
+        [Test]
+        public void TestBroadcastInOutException()
+        {
+            try
+            {
+                Grid1.Compute().Broadcast(Func(true), 1);
+
+                Assert.Fail();
+            }
+            catch (Exception e)
+            {
+                CheckError(e);
+            }
+        }
+
+        /// <summary>
+        /// Test apply in-out-closure execution.
+        /// </summary>
+        [Test]
+        public void TestApply()
+        {
+            object res = Grid1.Compute().Apply(Func(false), 1);
+
+            CheckResult(res);
+        }
+
+        /// <summary>
+        /// Test apply in-out-closure execution with exception.
+        /// </summary>
+        [Test]
+        public void TestApplyException()
+        {
+            try
+            {
+                Grid1.Compute().Apply(Func(true), 1);
+
+                Assert.Fail();
+            }
+            catch (Exception e)
+            {
+                CheckError(e);
+            }
+        }
+
+        /// <summary>
+        /// Test apply multiple in-out-closures execution.
+        /// </summary>
+        [Test]
+        public void TestApplyMultiple()
+        {
+            var args = new List<object>(MultiCloCnt);
+
+            for (int i = 0; i < MultiCloCnt; i++)
+                args.Add(1);
+
+            Console.WriteLine("START TASK");
+
+            var ress = Grid1.Compute().Apply(Func(false), args);
+
+            Console.WriteLine("END TASK.");
+
+            foreach (object res in ress)
+                CheckResult(res);
+        }
+
+        /// <summary>
+        /// Test apply multiple in-out-closures execution with exception.
+        /// </summary>
+        [Test]
+        public void TestApplyMultipleException()
+        {
+            ICollection<int> args = new List<int>(MultiCloCnt);
+
+            for (int i = 0; i < MultiCloCnt; i++)
+                args.Add(1);
+
+            try
+            {
+                Grid1.Compute().Apply(Func(true), args);
+
+                Assert.Fail();
+            }
+            catch (Exception e)
+            {
+                CheckError(e);
+            }
+        }
+
+        /// <summary>
+        /// Test apply multiple in-out-closures execution with reducer.
+        /// </summary>
+        [Test]
+        public void TestApplyMultipleReducer()
+        {
+            var args = new List<object>(MultiCloCnt);
+
+            for (int i = 0; i < MultiCloCnt; i++)
+                args.Add(1);
+
+            ICollection<object> ress =
+                Grid1.Compute().Apply(Func(false), args, new Reducer(false));
+
+            foreach (object res in ress)
+                CheckResult(res);
+        }
+
+        /// <summary>
+        /// Test apply multiple in-out-closures execution with reducer and exception thrown from closure.
+        /// </summary>
+        [Test]
+        public void TestAppylMultipleReducerJobException()
+        {
+            List<object> args = new List<object>(MultiCloCnt);
+
+            for (int i = 0; i < MultiCloCnt; i++)
+                args.Add(1);
+
+            try
+            {
+                Grid1.Compute().Apply(Func(true), args, new Reducer(false));
+
+                Assert.Fail();
+            }
+            catch (Exception e)
+            {
+                CheckError(e);
+            }
+        }
+
+        /// <summary>
+        /// Test apply multiple in-out-closures execution with reducer and exception thrown from reducer.
+        /// </summary>
+        [Test]
+        public void TestAppylMultipleReducerReduceException()
+        {
+            var args = new List<object>(MultiCloCnt);
+
+            for (int i = 0; i < MultiCloCnt; i++)
+                args.Add(1);
+
+            try
+            {
+                Grid1.Compute().Apply(Func(false), args, new Reducer(true));
+
+                Assert.Fail();
+            }
+            catch (Exception e)
+            {
+                Assert.AreEqual(typeof(Exception), e.GetType());
+
+                Assert.AreEqual(ErrMsg, e.Message);
+            }
+        }
+
+        /// <summary>
+        /// Create out-only closure.
+        /// </summary>
+        /// <param name="err">Error flag.</param>
+        /// <returns>Closure.</returns>
+        protected abstract IComputeFunc<object> OutFunc(bool err);
+
+        /// <summary>
+        /// Create in-out closure.
+        /// </summary>
+        /// <param name="err">Error flag.</param>
+        /// <returns>Closure.</returns>
+        protected abstract IComputeFunc<object, object> Func(bool err);
+
+        /// <summary>
+        /// Check result.
+        /// </summary>
+        /// <param name="res">Result.</param>
+        protected abstract void CheckResult(object res);
+
+        /// <summary>
+        /// Check error.
+        /// </summary>
+        /// <param name="err">Error.</param>
+        protected abstract void CheckError(Exception err);
+
+        /// <summary>
+        /// Test reducer.
+        /// </summary>
+        public class Reducer : IComputeReducer<object, ICollection<object>>
+        {
+            /** Whether to throw an error on reduce. */
+            private readonly bool _err;
+
+            /** Results. */
+            private readonly ICollection<object> _ress = new List<object>();
+
+            /// <summary>
+            /// Constructor.
+            /// </summary>
+            /// <param name="err">Error.</param>
+            public Reducer(bool err)
+            {
+                _err = err;
+            }
+
+            /** <inheritDoc /> */
+            public bool Collect(object res)
+            {
+                _ress.Add(res);
+
+                return true;
+            }
+
+            /** <inheritDoc /> */
+            public ICollection<object> Reduce()
+            {
+                if (_err)
+                    throw new Exception(ErrMsg);
+                return _ress;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs
new file mode 100644
index 0000000..75fc712
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs
@@ -0,0 +1,1281 @@
+/*
+ * 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 SpecifyACultureInStringConversionExplicitly
+namespace Apache.Ignite.Core.Tests.Compute
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Linq;
+    using System.Threading;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.Portable;
+    using Apache.Ignite.Core.Resource;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests for compute.
+    /// </summary>
+    public class ComputeApiTest
+    {
+        /** Echo task name. */
+        private const string EchoTask = "org.apache.ignite.platform.PlatformComputeEchoTask";
+
+        /** Portable argument task name. */
+        private const string PortableArgTask = "org.apache.ignite.platform.PlatformComputePortableArgTask";
+
+        /** Broadcast task name. */
+        private const string BroadcastTask = "org.apache.ignite.platform.PlatformComputeBroadcastTask";
+
+        /** Broadcast task name. */
+        private const string DecimalTask = "org.apache.ignite.platform.PlatformComputeDecimalTask";
+
+        /** Java portable class name. */
+        private const string JavaPortableCls = "GridInteropComputeJavaPortable";
+
+        /** Echo type: null. */
+        private const int EchoTypeNull = 0;
+
+        /** Echo type: byte. */
+        private const int EchoTypeByte = 1;
+
+        /** Echo type: bool. */
+        private const int EchoTypeBool = 2;
+
+        /** Echo type: short. */
+        private const int EchoTypeShort = 3;
+
+        /** Echo type: char. */
+        private const int EchoTypeChar = 4;
+
+        /** Echo type: int. */
+        private const int EchoTypeInt = 5;
+
+        /** Echo type: long. */
+        private const int EchoTypeLong = 6;
+
+        /** Echo type: float. */
+        private const int EchoTypeFloat = 7;
+
+        /** Echo type: double. */
+        private const int EchoTypeDouble = 8;
+
+        /** Echo type: array. */
+        private const int EchoTypeArray = 9;
+
+        /** Echo type: collection. */
+        private const int EchoTypeCollection = 10;
+
+        /** Echo type: map. */
+        private const int EchoTypeMap = 11;
+
+        /** Echo type: portable. */
+        private const int EchoTypePortable = 12;
+
+        /** Echo type: portable (Java only). */
+        private const int EchoTypePortableJava = 13;
+
+        /** Type: object array. */
+        private const int EchoTypeObjArray = 14;
+
+        /** Type: portable object array. */
+        private const int EchoTypePortableArray = 15;
+
+        /** Type: enum. */
+        private const int EchoTypeEnum = 16;
+
+        /** Type: enum array. */
+        private const int EchoTypeEnumArray = 17;
+
+        /** First node. */
+        private IIgnite _grid1;
+
+        /** Second node. */
+        private IIgnite _grid2;
+
+        /** Third node. */
+        private IIgnite _grid3;
+
+        /// <summary>
+        /// Initialization routine.
+        /// </summary>
+        [TestFixtureSetUp]
+        public void InitClient()
+        {
+            //TestUtils.JVM_DEBUG = true;
+            TestUtils.KillProcesses();
+
+            _grid1 = Ignition.Start(Configuration("config\\compute\\compute-grid1.xml"));
+            _grid2 = Ignition.Start(Configuration("config\\compute\\compute-grid2.xml"));
+            _grid3 = Ignition.Start(Configuration("config\\compute\\compute-grid3.xml"));
+        }
+
+        [TestFixtureTearDown]
+        public void StopClient()
+        {
+            if (_grid1 != null)
+                Ignition.Stop(_grid1.Name, true);
+
+            if (_grid2 != null)
+                Ignition.Stop(_grid2.Name, true);
+
+            if (_grid3 != null)
+                Ignition.Stop(_grid3.Name, true);
+        }
+
+        [TearDown]
+        public void AfterTest()
+        {
+            TestUtils.AssertHandleRegistryIsEmpty(1000, _grid1, _grid2, _grid3);
+        }
+
+        /// <summary>
+        /// Test that it is possible to get projection from grid.
+        /// </summary>
+        [Test]
+        public void TestProjection()
+        {
+            IClusterGroup prj = _grid1.Cluster;
+
+            Assert.NotNull(prj);
+
+            Assert.IsTrue(prj == prj.Ignite);
+        }
+
+        /// <summary>
+        /// Test getting cache with default (null) name.
+        /// </summary>
+        [Test]
+        public void TestCacheDefaultName()
+        {
+            var cache = _grid1.Cache<int, int>(null);
+
+            Assert.IsNotNull(cache);
+
+            cache.GetAndPut(1, 1);
+
+            Assert.AreEqual(1, cache.Get(1));
+        }
+
+        /// <summary>
+        /// Test non-existent cache.
+        /// </summary>
+        [Test]
+        public void TestNonExistentCache()
+        {
+            Assert.Catch(typeof(ArgumentException), () =>
+            {
+                _grid1.Cache<int, int>("bad_name");
+            });
+        }
+
+        /// <summary>
+        /// Test node content.
+        /// </summary>
+        [Test]
+        public void TestNodeContent()
+        {
+            ICollection<IClusterNode> nodes = _grid1.Cluster.Nodes();
+
+            foreach (IClusterNode node in nodes)
+            {
+                Assert.NotNull(node.Addresses);
+                Assert.IsTrue(node.Addresses.Count > 0);
+                Assert.Throws<NotSupportedException>(() => node.Addresses.Add("addr"));
+
+                Assert.NotNull(node.Attributes());
+                Assert.IsTrue(node.Attributes().Count > 0);
+                Assert.Throws<NotSupportedException>(() => node.Attributes().Add("key", "val"));
+
+                Assert.NotNull(node.HostNames);
+                Assert.Throws<NotSupportedException>(() => node.HostNames.Add("h"));
+
+                Assert.IsTrue(node.Id != Guid.Empty);
+
+                Assert.IsTrue(node.Order > 0);
+
+                Assert.NotNull(node.Metrics());
+            }
+        }
+
+        /// <summary>
+        /// Test cluster metrics.
+        /// </summary>
+        [Test]
+        public void TestClusterMetrics()
+        {
+            var cluster = _grid1.Cluster;
+
+            IClusterMetrics metrics = cluster.Metrics();
+
+            Assert.IsNotNull(metrics);
+
+            Assert.AreEqual(cluster.Nodes().Count, metrics.TotalNodes);
+
+            Thread.Sleep(2000);
+
+            IClusterMetrics newMetrics = cluster.Metrics();
+
+            Assert.IsFalse(metrics == newMetrics);
+            Assert.IsTrue(metrics.LastUpdateTime < newMetrics.LastUpdateTime);
+        }
+
+        /// <summary>
+        /// Test cluster metrics.
+        /// </summary>
+        [Test]
+        public void TestNodeMetrics()
+        {
+            var node = _grid1.Cluster.Node();
+
+            IClusterMetrics metrics = node.Metrics();
+
+            Assert.IsNotNull(metrics);
+
+            Assert.IsTrue(metrics == node.Metrics());
+
+            Thread.Sleep(2000);
+
+            IClusterMetrics newMetrics = node.Metrics();
+
+            Assert.IsFalse(metrics == newMetrics);
+            Assert.IsTrue(metrics.LastUpdateTime < newMetrics.LastUpdateTime);
+        }
+
+        /// <summary>
+        /// Test cluster metrics.
+        /// </summary>
+        [Test]
+        public void TestResetMetrics()
+        {
+            var cluster = _grid1.Cluster;
+
+            Thread.Sleep(2000);
+
+            var metrics1 = cluster.Metrics();
+
+            cluster.ResetMetrics();
+
+            var metrics2 = cluster.Metrics();
+
+            Assert.IsNotNull(metrics1);
+            Assert.IsNotNull(metrics2);
+        }
+
+        /// <summary>
+        /// Test node ping.
+        /// </summary>
+        [Test]
+        public void TestPingNode()
+        {
+            var cluster = _grid1.Cluster;
+
+            Assert.IsTrue(cluster.Nodes().Select(node => node.Id).All(cluster.PingNode));
+            
+            Assert.IsFalse(cluster.PingNode(Guid.NewGuid()));
+        }
+
+        /// <summary>
+        /// Tests the topology version.
+        /// </summary>
+        [Test]
+        public void TestTopologyVersion()
+        {
+            var cluster = _grid1.Cluster;
+            
+            var topVer = cluster.TopologyVersion;
+
+            Ignition.Stop(_grid3.Name, true);
+
+            Assert.AreEqual(topVer + 1, _grid1.Cluster.TopologyVersion);
+
+            _grid3 = Ignition.Start(Configuration("config\\compute\\compute-grid3.xml"));
+
+            Assert.AreEqual(topVer + 2, _grid1.Cluster.TopologyVersion);
+        }
+
+        /// <summary>
+        /// Tests the topology by version.
+        /// </summary>
+        [Test]
+        public void TestTopology()
+        {
+            var cluster = _grid1.Cluster;
+
+            Assert.AreEqual(1, cluster.Topology(1).Count);
+
+            Assert.AreEqual(null, cluster.Topology(int.MaxValue));
+
+            // Check that Nodes and Topology return the same for current version
+            var topVer = cluster.TopologyVersion;
+
+            var top = cluster.Topology(topVer);
+
+            var nodes = cluster.Nodes();
+
+            Assert.AreEqual(top.Count, nodes.Count);
+
+            Assert.IsTrue(top.All(nodes.Contains));
+
+            // Stop/start node to advance version and check that history is still correct
+            Assert.IsTrue(Ignition.Stop(_grid2.Name, true));
+
+            try
+            {
+                top = cluster.Topology(topVer);
+
+                Assert.AreEqual(top.Count, nodes.Count);
+
+                Assert.IsTrue(top.All(nodes.Contains));
+            }
+            finally 
+            {
+                _grid2 = Ignition.Start(Configuration("config\\compute\\compute-grid2.xml"));
+            }
+        }
+
+        /// <summary>
+        /// Test nodes in full topology.
+        /// </summary>
+        [Test]
+        public void TestNodes()
+        {
+            Assert.IsNotNull(_grid1.Cluster.Node());
+
+            ICollection<IClusterNode> nodes = _grid1.Cluster.Nodes();
+
+            Assert.IsTrue(nodes.Count == 3);
+
+            // Check subsequent call on the same topology.
+            nodes = _grid1.Cluster.Nodes();
+
+            Assert.IsTrue(nodes.Count == 3);
+
+            Assert.IsTrue(Ignition.Stop(_grid2.Name, true));
+
+            // Check subsequent calls on updating topologies.
+            nodes = _grid1.Cluster.Nodes();
+
+            Assert.IsTrue(nodes.Count == 2);
+
+            nodes = _grid1.Cluster.Nodes();
+
+            Assert.IsTrue(nodes.Count == 2);
+
+            _grid2 = Ignition.Start(Configuration("config\\compute\\compute-grid2.xml"));
+
+            nodes = _grid1.Cluster.Nodes();
+
+            Assert.IsTrue(nodes.Count == 3);
+        }
+
+        /// <summary>
+        /// Test "ForNodes" and "ForNodeIds".
+        /// </summary>
+        [Test]
+        public void TestForNodes()
+        {
+            ICollection<IClusterNode> nodes = _grid1.Cluster.Nodes();
+
+            IClusterNode first = nodes.ElementAt(0);
+            IClusterNode second = nodes.ElementAt(1);
+
+            IClusterGroup singleNodePrj = _grid1.Cluster.ForNodeIds(first.Id);
+            Assert.AreEqual(1, singleNodePrj.Nodes().Count);
+            Assert.AreEqual(first.Id, singleNodePrj.Nodes().First().Id);
+
+            singleNodePrj = _grid1.Cluster.ForNodeIds(new List<Guid> { first.Id });
+            Assert.AreEqual(1, singleNodePrj.Nodes().Count);
+            Assert.AreEqual(first.Id, singleNodePrj.Nodes().First().Id);
+
+            singleNodePrj = _grid1.Cluster.ForNodes(first);
+            Assert.AreEqual(1, singleNodePrj.Nodes().Count);
+            Assert.AreEqual(first.Id, singleNodePrj.Nodes().First().Id);
+
+            singleNodePrj = _grid1.Cluster.ForNodes(new List<IClusterNode> { first });
+            Assert.AreEqual(1, singleNodePrj.Nodes().Count);
+            Assert.AreEqual(first.Id, singleNodePrj.Nodes().First().Id);
+
+            IClusterGroup multiNodePrj = _grid1.Cluster.ForNodeIds(first.Id, second.Id);
+            Assert.AreEqual(2, multiNodePrj.Nodes().Count);
+            Assert.IsTrue(multiNodePrj.Nodes().Contains(first));
+            Assert.IsTrue(multiNodePrj.Nodes().Contains(second));
+
+            multiNodePrj = _grid1.Cluster.ForNodeIds(new[] {first, second}.Select(x => x.Id));
+            Assert.AreEqual(2, multiNodePrj.Nodes().Count);
+            Assert.IsTrue(multiNodePrj.Nodes().Contains(first));
+            Assert.IsTrue(multiNodePrj.Nodes().Contains(second));
+
+            multiNodePrj = _grid1.Cluster.ForNodes(first, second);
+            Assert.AreEqual(2, multiNodePrj.Nodes().Count);
+            Assert.IsTrue(multiNodePrj.Nodes().Contains(first));
+            Assert.IsTrue(multiNodePrj.Nodes().Contains(second));
+
+            multiNodePrj = _grid1.Cluster.ForNodes(new List<IClusterNode> { first, second });
+            Assert.AreEqual(2, multiNodePrj.Nodes().Count);
+            Assert.IsTrue(multiNodePrj.Nodes().Contains(first));
+            Assert.IsTrue(multiNodePrj.Nodes().Contains(second));
+        }
+
+        /// <summary>
+        /// Test "ForNodes" and "ForNodeIds". Make sure lazy enumerables are enumerated only once.
+        /// </summary>
+        [Test]
+        public void TestForNodesLaziness()
+        {
+            var nodes = _grid1.Cluster.Nodes().Take(2).ToArray();
+
+            var callCount = 0;
+            
+            Func<IClusterNode, IClusterNode> nodeSelector = node =>
+            {
+                callCount++;
+                return node;
+            };
+
+            Func<IClusterNode, Guid> idSelector = node =>
+            {
+                callCount++;
+                return node.Id;
+            };
+
+            var projection = _grid1.Cluster.ForNodes(nodes.Select(nodeSelector));
+            Assert.AreEqual(2, projection.Nodes().Count);
+            Assert.AreEqual(2, callCount);
+            
+            projection = _grid1.Cluster.ForNodeIds(nodes.Select(idSelector));
+            Assert.AreEqual(2, projection.Nodes().Count);
+            Assert.AreEqual(4, callCount);
+        }
+
+        /// <summary>
+        /// Test for local node projection.
+        /// </summary>
+        [Test]
+        public void TestForLocal()
+        {
+            IClusterGroup prj = _grid1.Cluster.ForLocal();
+
+            Assert.AreEqual(1, prj.Nodes().Count);
+            Assert.AreEqual(_grid1.Cluster.LocalNode, prj.Nodes().First());
+        }
+
+        /// <summary>
+        /// Test for remote nodes projection.
+        /// </summary>
+        [Test]
+        public void TestForRemotes()
+        {
+            ICollection<IClusterNode> nodes = _grid1.Cluster.Nodes();
+
+            IClusterGroup prj = _grid1.Cluster.ForRemotes();
+
+            Assert.AreEqual(2, prj.Nodes().Count);
+            Assert.IsTrue(nodes.Contains(prj.Nodes().ElementAt(0)));
+            Assert.IsTrue(nodes.Contains(prj.Nodes().ElementAt(1)));
+        }
+
+        /// <summary>
+        /// Test for host nodes projection.
+        /// </summary>
+        [Test]
+        public void TestForHost()
+        {
+            ICollection<IClusterNode> nodes = _grid1.Cluster.Nodes();
+
+            IClusterGroup prj = _grid1.Cluster.ForHost(nodes.First());
+
+            Assert.AreEqual(3, prj.Nodes().Count);
+            Assert.IsTrue(nodes.Contains(prj.Nodes().ElementAt(0)));
+            Assert.IsTrue(nodes.Contains(prj.Nodes().ElementAt(1)));
+            Assert.IsTrue(nodes.Contains(prj.Nodes().ElementAt(2)));
+        }
+
+        /// <summary>
+        /// Test for oldest, youngest and random projections.
+        /// </summary>
+        [Test]
+        public void TestForOldestYoungestRandom()
+        {
+            ICollection<IClusterNode> nodes = _grid1.Cluster.Nodes();
+
+            IClusterGroup prj = _grid1.Cluster.ForYoungest();
+            Assert.AreEqual(1, prj.Nodes().Count);
+            Assert.IsTrue(nodes.Contains(prj.Node()));
+
+            prj = _grid1.Cluster.ForOldest();
+            Assert.AreEqual(1, prj.Nodes().Count);
+            Assert.IsTrue(nodes.Contains(prj.Node()));
+
+            prj = _grid1.Cluster.ForRandom();
+            Assert.AreEqual(1, prj.Nodes().Count);
+            Assert.IsTrue(nodes.Contains(prj.Node()));
+        }
+
+        /// <summary>
+        /// Test for attribute projection.
+        /// </summary>
+        [Test]
+        public void TestForAttribute()
+        {
+            ICollection<IClusterNode> nodes = _grid1.Cluster.Nodes();
+
+            IClusterGroup prj = _grid1.Cluster.ForAttribute("my_attr", "value1");
+            Assert.AreEqual(1, prj.Nodes().Count);
+            Assert.IsTrue(nodes.Contains(prj.Node()));
+            Assert.AreEqual("value1", prj.Nodes().First().Attribute<string>("my_attr"));
+        }
+        
+        /// <summary>
+        /// Test for cache/data/client projections.
+        /// </summary>
+        [Test]
+        public void TestForCacheNodes()
+        {
+            ICollection<IClusterNode> nodes = _grid1.Cluster.Nodes();
+
+            // Cache nodes.
+            IClusterGroup prjCache = _grid1.Cluster.ForCacheNodes("cache1");
+
+            Assert.AreEqual(2, prjCache.Nodes().Count);
+
+            Assert.IsTrue(nodes.Contains(prjCache.Nodes().ElementAt(0)));
+            Assert.IsTrue(nodes.Contains(prjCache.Nodes().ElementAt(1)));
+            
+            // Data nodes.
+            IClusterGroup prjData = _grid1.Cluster.ForDataNodes("cache1");
+
+            Assert.AreEqual(2, prjData.Nodes().Count);
+
+            Assert.IsTrue(prjCache.Nodes().Contains(prjData.Nodes().ElementAt(0)));
+            Assert.IsTrue(prjCache.Nodes().Contains(prjData.Nodes().ElementAt(1)));
+
+            // Client nodes.
+            IClusterGroup prjClient = _grid1.Cluster.ForClientNodes("cache1");
+
+            Assert.AreEqual(0, prjClient.Nodes().Count);
+        }
+        
+        /// <summary>
+        /// Test for cache predicate.
+        /// </summary>
+        [Test]
+        public void TestForPredicate()
+        {
+            IClusterGroup prj1 = _grid1.Cluster.ForPredicate(new NotAttributePredicate("value1").Apply);
+            Assert.AreEqual(2, prj1.Nodes().Count);
+
+            IClusterGroup prj2 = prj1.ForPredicate(new NotAttributePredicate("value2").Apply);
+            Assert.AreEqual(1, prj2.Nodes().Count);
+
+            string val;
+
+            prj2.Nodes().First().TryGetAttribute("my_attr", out val);
+
+            Assert.IsTrue(val == null || (!val.Equals("value1") && !val.Equals("value2")));
+        }
+
+        /// <summary>
+        /// Attribute predicate.
+        /// </summary>
+        private class NotAttributePredicate
+        {
+            /** Required attribute value. */
+            private readonly string _attrVal;
+
+            /// <summary>
+            /// Constructor.
+            /// </summary>
+            /// <param name="attrVal">Required attribute value.</param>
+            public NotAttributePredicate(string attrVal)
+            {
+                _attrVal = attrVal;
+            }
+
+            /** <inhreitDoc /> */
+            public bool Apply(IClusterNode node)
+            {
+                string val;
+
+                node.TryGetAttribute("my_attr", out val);
+
+                return val == null || !val.Equals(_attrVal);
+            }
+        }
+
+        /// <summary>
+        /// Test echo with decimals.
+        /// </summary>
+        [Test]
+        public void TestEchoDecimal()
+        {
+            decimal val;
+
+            Assert.AreEqual(val = decimal.Zero, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+
+            Assert.AreEqual(val = new decimal(0, 0, 1, false, 0), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, 0, 1, true, 0), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, 0, 1, false, 0) - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, 0, 1, true, 0) - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, 0, 1, false, 0) + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, 0, 1, true, 0) + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, 0, int.MinValue, false, 0), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, 0, int.MinValue, true, 0), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, 0, int.MinValue, false, 0) - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, 0, int.MinValue, true, 0) - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, 0, int.MinValue, false, 0) + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, 0, int.MinValue, true, 0) + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, 0, int.MaxValue, false, 0), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, 0, int.MaxValue, true, 0), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, 0, int.MaxValue, false, 0) - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, 0, int.MaxValue, true, 0) - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, 0, int.MaxValue, false, 0) + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, 0, int.MaxValue, true, 0) + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+
+            Assert.AreEqual(val = new decimal(0, 1, 0, false, 0), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, 1, 0, true, 0), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, 1, 0, false, 0) - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, 1, 0, true, 0) - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, 1, 0, false, 0) + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, 1, 0, true, 0) + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, int.MinValue, 0, false, 0), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, int.MinValue, 0, true, 0), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, int.MinValue, 0, false, 0) - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, int.MinValue, 0, true, 0) - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, int.MinValue, 0, false, 0) + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, int.MinValue, 0, true, 0) + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, int.MaxValue, 0, false, 0), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, int.MaxValue, 0, true, 0), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, int.MaxValue, 0, false, 0) - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, int.MaxValue, 0, true, 0) - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, int.MaxValue, 0, false, 0) + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(0, int.MaxValue, 0, true, 0) + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+
+            Assert.AreEqual(val = new decimal(1, 0, 0, false, 0), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(1, 0, 0, true, 0), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(1, 0, 0, false, 0) - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(1, 0, 0, true, 0) - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(1, 0, 0, false, 0) + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(1, 0, 0, true, 0) + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(int.MinValue, 0, 0, false, 0), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(int.MinValue, 0, 0, true, 0), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(int.MinValue, 0, 0, false, 0) - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(int.MinValue, 0, 0, true, 0) - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(int.MinValue, 0, 0, false, 0) + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(int.MinValue, 0, 0, true, 0) + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(int.MaxValue, 0, 0, false, 0), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(int.MaxValue, 0, 0, true, 0), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(int.MaxValue, 0, 0, false, 0) - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(int.MaxValue, 0, 0, true, 0) - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(int.MaxValue, 0, 0, false, 0) + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(int.MaxValue, 0, 0, true, 0) + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+
+            Assert.AreEqual(val = new decimal(1, 1, 1, false, 0), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(1, 1, 1, true, 0), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(1, 1, 1, false, 0) - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(1, 1, 1, true, 0) - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(1, 1, 1, false, 0) + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = new decimal(1, 1, 1, true, 0) + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+
+            Assert.AreEqual(val = decimal.Parse("65536"), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.Parse("-65536"), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.Parse("65536") - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.Parse("-65536") - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.Parse("65536") + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.Parse("-65536") + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+
+            Assert.AreEqual(val = decimal.Parse("4294967296"), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.Parse("-4294967296"), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.Parse("4294967296") - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.Parse("-4294967296") - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.Parse("4294967296") + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.Parse("-4294967296") + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+
+            Assert.AreEqual(val = decimal.Parse("281474976710656"), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.Parse("-281474976710656"), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.Parse("281474976710656") - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.Parse("-281474976710656") - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.Parse("281474976710656") + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.Parse("-281474976710656") + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+
+            Assert.AreEqual(val = decimal.Parse("18446744073709551616"), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.Parse("-18446744073709551616"), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.Parse("18446744073709551616") - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.Parse("-18446744073709551616") - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.Parse("18446744073709551616") + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.Parse("-18446744073709551616") + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+
+            Assert.AreEqual(val = decimal.Parse("1208925819614629174706176"), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.Parse("-1208925819614629174706176"), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.Parse("1208925819614629174706176") - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.Parse("-1208925819614629174706176") - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.Parse("1208925819614629174706176") + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.Parse("-1208925819614629174706176") + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+
+            Assert.AreEqual(val = decimal.MaxValue, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.MinValue, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.MaxValue - 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.MinValue + 1, _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+
+            Assert.AreEqual(val = decimal.Parse("11,12"), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+            Assert.AreEqual(val = decimal.Parse("-11,12"), _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { val, val.ToString() }));
+
+            // Test echo with overflow.
+            try
+            {
+                _grid1.Compute().ExecuteJavaTask<object>(DecimalTask, new object[] { null, decimal.MaxValue.ToString() + 1 });
+
+                Assert.Fail();
+            }
+            catch (IgniteException)
+            {
+                // No-op.
+            }
+        }
+
+        /// <summary>
+        /// Test echo task returning null.
+        /// </summary>
+        [Test]
+        public void TestEchoTaskNull()
+        {
+            Assert.IsNull(_grid1.Compute().ExecuteJavaTask<object>(EchoTask, EchoTypeNull));
+        }
+
+        /// <summary>
+        /// Test echo task returning various primitives.
+        /// </summary>
+        [Test]
+        public void TestEchoTaskPrimitives()
+        {
+            Assert.AreEqual(1, _grid1.Compute().ExecuteJavaTask<byte>(EchoTask, EchoTypeByte));
+            Assert.AreEqual(true, _grid1.Compute().ExecuteJavaTask<bool>(EchoTask, EchoTypeBool));
+            Assert.AreEqual(1, _grid1.Compute().ExecuteJavaTask<short>(EchoTask, EchoTypeShort));
+            Assert.AreEqual((char)1, _grid1.Compute().ExecuteJavaTask<char>(EchoTask, EchoTypeChar));
+            Assert.AreEqual(1, _grid1.Compute().ExecuteJavaTask<int>(EchoTask, EchoTypeInt));
+            Assert.AreEqual(1, _grid1.Compute().ExecuteJavaTask<long>(EchoTask, EchoTypeLong));
+            Assert.AreEqual((float)1, _grid1.Compute().ExecuteJavaTask<float>(EchoTask, EchoTypeFloat));
+            Assert.AreEqual((double)1, _grid1.Compute().ExecuteJavaTask<double>(EchoTask, EchoTypeDouble));
+        }
+
+        /// <summary>
+        /// Test echo task returning compound types.
+        /// </summary>
+        [Test]
+        public void TestEchoTaskCompound()
+        {
+            int[] res1 = _grid1.Compute().ExecuteJavaTask<int[]>(EchoTask, EchoTypeArray);
+
+            Assert.AreEqual(1, res1.Length);
+            Assert.AreEqual(1, res1[0]);
+
+            IList<int> res2 = _grid1.Compute().ExecuteJavaTask<IList<int>>(EchoTask, EchoTypeCollection);
+
+            Assert.AreEqual(1, res2.Count);
+            Assert.AreEqual(1, res2[0]);
+
+            IDictionary<int, int> res3 = _grid1.Compute().ExecuteJavaTask<IDictionary<int, int>>(EchoTask, EchoTypeMap);
+
+            Assert.AreEqual(1, res3.Count);
+            Assert.AreEqual(1, res3[1]);
+        }
+
+        /// <summary>
+        /// Test echo task returning portable object.
+        /// </summary>
+        [Test]
+        public void TestEchoTaskPortable()
+        {
+            PlatformComputePortable res = _grid1.Compute().ExecuteJavaTask<PlatformComputePortable>(EchoTask, EchoTypePortable);
+
+            Assert.AreEqual(1, res.Field);
+        }
+
+        /// <summary>
+        /// Test echo task returning portable object with no corresponding class definition.
+        /// </summary>
+        [Test]
+        public void TestEchoTaskPortableNoClass()
+        {
+            ICompute compute = _grid1.Compute();
+
+            compute.WithKeepPortable();
+
+            IPortableObject res = compute.ExecuteJavaTask<IPortableObject>(EchoTask, EchoTypePortableJava);
+
+            Assert.AreEqual(1, res.Field<int>("field"));
+
+            // This call must fail because "keepPortable" flag is reset.
+            Assert.Catch(typeof(PortableException), () =>
+            {
+                compute.ExecuteJavaTask<IPortableObject>(EchoTask, EchoTypePortableJava);
+            });
+        }
+
+        /// <summary>
+        /// Tests the echo task returning object array.
+        /// </summary>
+        [Test]
+        public void TestEchoTaskObjectArray()
+        {
+            var res = _grid1.Compute().ExecuteJavaTask<string[]>(EchoTask, EchoTypeObjArray);
+            
+            Assert.AreEqual(new[] {"foo", "bar", "baz"}, res);
+        }
+
+        /// <summary>
+        /// Tests the echo task returning portable array.
+        /// </summary>
+        [Test]
+        public void TestEchoTaskPortableArray()
+        {
+            var res = _grid1.Compute().ExecuteJavaTask<PlatformComputePortable[]>(EchoTask, EchoTypePortableArray);
+            
+            Assert.AreEqual(3, res.Length);
+
+            for (var i = 0; i < res.Length; i++)
+                Assert.AreEqual(i + 1, res[i].Field);
+        }
+
+        /// <summary>
+        /// Tests the echo task returning enum.
+        /// </summary>
+        [Test]
+        public void TestEchoTaskEnum()
+        {
+            var res = _grid1.Compute().ExecuteJavaTask<InteropComputeEnum>(EchoTask, EchoTypeEnum);
+
+            Assert.AreEqual(InteropComputeEnum.Bar, res);
+        }
+
+        /// <summary>
+        /// Tests the echo task returning enum.
+        /// </summary>
+        [Test]
+        public void TestEchoTaskEnumArray()
+        {
+            var res = _grid1.Compute().ExecuteJavaTask<InteropComputeEnum[]>(EchoTask, EchoTypeEnumArray);
+
+            Assert.AreEqual(new[]
+            {
+                InteropComputeEnum.Bar,
+                InteropComputeEnum.Baz,
+                InteropComputeEnum.Foo
+            }, res);
+        }
+
+        /// <summary>
+        /// Test for portable argument in Java.
+        /// </summary>
+        [Test]
+        public void TestPortableArgTask()
+        {
+            ICompute compute = _grid1.Compute();
+
+            compute.WithKeepPortable();
+
+            PlatformComputeNetPortable arg = new PlatformComputeNetPortable();
+
+            arg.Field = 100;
+
+            int res = compute.ExecuteJavaTask<int>(PortableArgTask, arg);
+
+            Assert.AreEqual(arg.Field, res);
+        }
+
+        /// <summary>
+        /// Test running broadcast task.
+        /// </summary>
+        [Test]
+        public void TestBroadcastTask()
+        {
+            ICollection<Guid> res = _grid1.Compute().ExecuteJavaTask<ICollection<Guid>>(BroadcastTask, null);
+
+            Assert.AreEqual(3, res.Count);
+            Assert.AreEqual(1, _grid1.Cluster.ForNodeIds(res.ElementAt(0)).Nodes().Count);
+            Assert.AreEqual(1, _grid1.Cluster.ForNodeIds(res.ElementAt(1)).Nodes().Count);
+            Assert.AreEqual(1, _grid1.Cluster.ForNodeIds(res.ElementAt(2)).Nodes().Count);
+
+            var prj = _grid1.Cluster.ForPredicate(node => res.Take(2).Contains(node.Id));
+
+            Assert.AreEqual(2, prj.Nodes().Count);
+
+            ICollection<Guid> filteredRes = prj.Compute().ExecuteJavaTask<ICollection<Guid>>(BroadcastTask, null);
+
+            Assert.AreEqual(2, filteredRes.Count);
+            Assert.IsTrue(filteredRes.Contains(res.ElementAt(0)));
+            Assert.IsTrue(filteredRes.Contains(res.ElementAt(1)));
+        }
+
+        /// <summary>
+        /// Test running broadcast task in async mode.
+        /// </summary>
+        [Test]
+        public void TestBroadcastTaskAsync()
+        {
+            var gridCompute = _grid1.Compute().WithAsync();
+            Assert.IsNull(gridCompute.ExecuteJavaTask<ICollection<Guid>>(BroadcastTask, null));
+            ICollection<Guid> res = gridCompute.GetFuture<ICollection<Guid>>().Get();
+
+            Assert.AreEqual(3, res.Count);
+            Assert.AreEqual(1, _grid1.Cluster.ForNodeIds(res.ElementAt(0)).Nodes().Count);
+            Assert.AreEqual(1, _grid1.Cluster.ForNodeIds(res.ElementAt(1)).Nodes().Count);
+            Assert.AreEqual(1, _grid1.Cluster.ForNodeIds(res.ElementAt(2)).Nodes().Count);
+
+            var prj = _grid1.Cluster.ForPredicate(node => res.Take(2).Contains(node.Id));
+
+            Assert.AreEqual(2, prj.Nodes().Count);
+
+            var compute = prj.Compute().WithAsync();
+            Assert.IsNull(compute.ExecuteJavaTask<ICollection<Guid>>(BroadcastTask, null));
+            ICollection<Guid> filteredRes = compute.GetFuture<ICollection<Guid>>().Get();
+
+            Assert.AreEqual(2, filteredRes.Count);
+            Assert.IsTrue(filteredRes.Contains(res.ElementAt(0)));
+            Assert.IsTrue(filteredRes.Contains(res.ElementAt(1)));
+        }
+
+        /// <summary>
+        /// Tests the action broadcast.
+        /// </summary>
+        [Test]
+        public void TestBroadcastAction()
+        {
+            ComputeAction.InvokeCount = 0;
+            
+            _grid1.Compute().Broadcast(new ComputeAction());
+
+            Assert.AreEqual(_grid1.Cluster.Nodes().Count, ComputeAction.InvokeCount);
+        }
+
+        /// <summary>
+        /// Tests single action run.
+        /// </summary>
+        [Test]
+        public void TestRunAction()
+        {
+            ComputeAction.InvokeCount = 0;
+            
+            _grid1.Compute().Run(new ComputeAction());
+
+            Assert.AreEqual(1, ComputeAction.InvokeCount);
+        }
+
+        /// <summary>
+        /// Tests multiple actions run.
+        /// </summary>
+        [Test]
+        public void TestRunActions()
+        {
+            ComputeAction.InvokeCount = 0;
+
+            var actions = Enumerable.Range(0, 10).Select(x => new ComputeAction());
+            
+            _grid1.Compute().Run(actions);
+
+            Assert.AreEqual(10, ComputeAction.InvokeCount);
+        }
+
+        /// <summary>
+        /// Tests affinity run.
+        /// </summary>
+        [Test]
+        public void TestAffinityRun()
+        {
+            const string cacheName = null;
+
+            // Test keys for non-client nodes
+            var nodes = new[] {_grid1, _grid2}.Select(x => x.Cluster.LocalNode);
+
+            var aff = _grid1.Affinity(cacheName);
+
+            foreach (var node in nodes)
+            {
+                var primaryKey = Enumerable.Range(1, int.MaxValue).First(x => aff.IsPrimary(node, x));
+
+                var affinityKey = _grid1.Affinity(cacheName).AffinityKey<int, int>(primaryKey);
+
+                _grid1.Compute().AffinityRun(cacheName, affinityKey, new ComputeAction());
+
+                Assert.AreEqual(node.Id, ComputeAction.LastNodeId);
+            }
+        }
+
+        /// <summary>
+        /// Tests affinity call.
+        /// </summary>
+        [Test]
+        public void TestAffinityCall()
+        {
+            const string cacheName = null;
+
+            // Test keys for non-client nodes
+            var nodes = new[] { _grid1, _grid2 }.Select(x => x.Cluster.LocalNode);
+
+            var aff = _grid1.Affinity(cacheName);
+
+            foreach (var node in nodes)
+            {
+                var primaryKey = Enumerable.Range(1, int.MaxValue).First(x => aff.IsPrimary(node, x));
+
+                var affinityKey = _grid1.Affinity(cacheName).AffinityKey<int, int>(primaryKey);
+
+                var result = _grid1.Compute().AffinityCall(cacheName, affinityKey, new ComputeFunc());
+
+                Assert.AreEqual(result, ComputeFunc.InvokeCount);
+
+                Assert.AreEqual(node.Id, ComputeFunc.LastNodeId);
+            }
+        }
+
+        /// <summary>
+        /// Test "withNoFailover" feature.
+        /// </summary>
+        [Test]
+        public void TestWithNoFailover()
+        {
+            ICollection<Guid> res = _grid1.Compute().WithNoFailover().ExecuteJavaTask<ICollection<Guid>>(BroadcastTask, null);
+
+            Assert.AreEqual(3, res.Count);
+            Assert.AreEqual(1, _grid1.Cluster.ForNodeIds(res.ElementAt(0)).Nodes().Count);
+            Assert.AreEqual(1, _grid1.Cluster.ForNodeIds(res.ElementAt(1)).Nodes().Count);
+            Assert.AreEqual(1, _grid1.Cluster.ForNodeIds(res.ElementAt(2)).Nodes().Count);
+        }
+
+        /// <summary>
+        /// Test "withTimeout" feature.
+        /// </summary>
+        [Test]
+        public void TestWithTimeout()
+        {
+            ICollection<Guid> res = _grid1.Compute().WithTimeout(1000).ExecuteJavaTask<ICollection<Guid>>(BroadcastTask, null);
+
+            Assert.AreEqual(3, res.Count);
+            Assert.AreEqual(1, _grid1.Cluster.ForNodeIds(res.ElementAt(0)).Nodes().Count);
+            Assert.AreEqual(1, _grid1.Cluster.ForNodeIds(res.ElementAt(1)).Nodes().Count);
+            Assert.AreEqual(1, _grid1.Cluster.ForNodeIds(res.ElementAt(2)).Nodes().Count);
+        }
+
+        /// <summary>
+        /// Test simple dotNet task execution.
+        /// </summary>
+        [Test]
+        public void TestNetTaskSimple()
+        {
+            int res = _grid1.Compute().Execute<NetSimpleJobArgument, NetSimpleJobResult, NetSimpleTaskResult>(
+                    typeof(NetSimpleTask), new NetSimpleJobArgument(1)).Res;
+
+            Assert.AreEqual(_grid1.Compute().ClusterGroup.Nodes().Count, res);
+        }
+
+        /// <summary>
+        /// Create configuration.
+        /// </summary>
+        /// <param name="path">XML config path.</param>
+        private IgniteConfiguration Configuration(string path)
+        {
+            IgniteConfiguration cfg = new IgniteConfiguration();
+
+            PortableConfiguration portCfg = new PortableConfiguration();
+
+            ICollection<PortableTypeConfiguration> portTypeCfgs = new List<PortableTypeConfiguration>();
+
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(PlatformComputePortable)));
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(PlatformComputeNetPortable)));
+            portTypeCfgs.Add(new PortableTypeConfiguration(JavaPortableCls));
+
+            portCfg.TypeConfigurations = portTypeCfgs;
+
+            cfg.PortableConfiguration = portCfg;
+
+            cfg.JvmClasspath = IgniteManager.CreateClasspath(cfg, true);
+
+            cfg.JvmOptions = TestUtils.TestJavaOptions();
+
+            cfg.SpringConfigUrl = path;
+
+            return cfg;
+        }
+    }
+
+    class PlatformComputePortable
+    {
+        public int Field
+        {
+            get;
+            set;
+        }
+    }
+
+    class PlatformComputeNetPortable : PlatformComputePortable
+    {
+
+    }
+
+    [Serializable]
+    class NetSimpleTask : IComputeTask<NetSimpleJobArgument, NetSimpleJobResult, NetSimpleTaskResult>
+    {
+        /** <inheritDoc /> */
+
+        public IDictionary<IComputeJob<NetSimpleJobResult>, IClusterNode> Map(IList<IClusterNode> subgrid,
+            NetSimpleJobArgument arg)
+        {
+            var jobs = new Dictionary<IComputeJob<NetSimpleJobResult>, IClusterNode>();
+
+            for (int i = 0; i < subgrid.Count; i++)
+            {
+                NetSimpleJob job = new NetSimpleJob {Arg = arg};
+
+                jobs[job] = subgrid[i];
+            }
+
+            return jobs;
+        }
+
+        /** <inheritDoc /> */
+        public ComputeJobResultPolicy Result(IComputeJobResult<NetSimpleJobResult> res,
+            IList<IComputeJobResult<NetSimpleJobResult>> rcvd)
+        {
+            return ComputeJobResultPolicy.Wait;
+        }
+
+        /** <inheritDoc /> */
+        public NetSimpleTaskResult Reduce(IList<IComputeJobResult<NetSimpleJobResult>> results)
+        {
+            return new NetSimpleTaskResult(results.Sum(res => res.Data().Res));
+        }
+    }
+
+    [Serializable]
+    class NetSimpleJob : IComputeJob<NetSimpleJobResult>
+    {
+        public NetSimpleJobArgument Arg;
+
+        /** <inheritDoc /> */
+        public NetSimpleJobResult Execute()
+        {
+            return new NetSimpleJobResult(Arg.Arg);
+        }
+
+        /** <inheritDoc /> */
+        public void Cancel()
+        {
+            // No-op.
+        }
+    }
+
+    [Serializable]
+    class NetSimpleJobArgument
+    {
+        public int Arg;
+
+        public NetSimpleJobArgument(int arg)
+        {
+            Arg = arg;
+        }
+    }
+
+    [Serializable]
+    class NetSimpleTaskResult
+    {
+        public int Res;
+
+        public NetSimpleTaskResult(int res)
+        {
+            Res = res;
+        }
+    }
+
+    [Serializable]
+    class NetSimpleJobResult
+    {
+        public int Res;
+
+        public NetSimpleJobResult(int res)
+        {
+            Res = res;
+        }
+    }
+
+    [Serializable]
+    class ComputeAction : IComputeAction
+    {
+        [InstanceResource]
+        #pragma warning disable 649
+        private IIgnite _grid;
+
+        public static int InvokeCount;
+
+        public static Guid LastNodeId;
+
+        public void Invoke()
+        {
+            Interlocked.Increment(ref InvokeCount);
+            LastNodeId = _grid.Cluster.LocalNode.Id;
+        }
+    }
+
+    interface IUserInterface<out T>
+    {
+        T Invoke();
+    }
+
+    interface INestedComputeFunc : IComputeFunc<int>
+    {
+        
+    }
+
+    [Serializable]
+    class ComputeFunc : INestedComputeFunc, IUserInterface<int>
+    {
+        [InstanceResource]
+        private IIgnite _grid;
+
+        public static int InvokeCount;
+
+        public static Guid LastNodeId;
+
+        int IComputeFunc<int>.Invoke()
+        {
+            InvokeCount++;
+            LastNodeId = _grid.Cluster.LocalNode.Id;
+            return InvokeCount;
+        }
+
+        int IUserInterface<int>.Invoke()
+        {
+            // Same signature as IComputeFunc<int>, but from different interface
+            throw new Exception("Invalid method");
+        }
+
+        public int Invoke()
+        {
+            // Same signature as IComputeFunc<int>, but due to explicit interface implementation this is a wrong method
+            throw new Exception("Invalid method");
+        }
+    }
+
+    public enum InteropComputeEnum
+    {
+        Foo,
+        Bar,
+        Baz
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeMultithreadedTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeMultithreadedTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeMultithreadedTest.cs
new file mode 100644
index 0000000..36a0505
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeMultithreadedTest.cs
@@ -0,0 +1,269 @@
+/*
+ * 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.Compute
+{
+    using System;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Compute;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests class.
+    /// </summary>
+    [Category(TestUtils.CategoryIntensive)]
+    public class ComputeMultithreadedTest : AbstractTaskTest
+    {
+        /** */
+        private static IList<Action<ICompute>> _actions;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        public ComputeMultithreadedTest() : base(false) { }
+
+        /// <summary>
+        /// Set-up routine.
+        /// </summary>
+        [SetUp]
+        public void SetUp()
+        {
+            _actions = new List<Action<ICompute>>
+            {
+                compute => { compute.Apply(new My1ArgClosure(), "zzzz"); },
+                compute => { compute.Broadcast(new My1ArgClosure(), "zzzz"); },
+                compute => { compute.Broadcast(new MyNoArgClosure("zzzz")); },
+                compute => { compute.Call(new MyNoArgClosure("zzzz")); },
+                compute => { compute.Execute(new StringLengthEmptyTask(), "zzzz"); },
+                compute =>
+                {
+                    compute.Apply(new My1ArgClosure(), new List<string> {"zzzz", "a", "b"}, new MyReducer());
+                }
+            };
+
+        }
+
+        /// <summary>
+        /// Tear-down routine.
+        /// </summary>
+        [TearDown]
+        public void TearDown()
+        {
+            _actions.Clear();
+        }
+
+        /// <summary>
+        /// Test not-marshalable error occurred during map step.
+        /// </summary>
+        [Test]
+        public void TestAllTaskTypeAtSameTime()
+        {
+            Assert.AreEqual(_actions.Count, 6);
+
+            var compute = Grid1.Compute();
+
+            TestUtils.RunMultiThreaded(() =>
+            {
+                _actions[TestUtils.Random.Next(_actions.Count)](compute);
+            }, 4, 60);
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        [Test]
+        public void TestSingleTaskType0()
+        {
+            Assert.AreEqual(_actions.Count, 6);
+
+            TestUtils.RunMultiThreaded(() => _actions[0](Grid1.Compute()), 4, 20);
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        [Test]
+        public void TestSingleTaskType1()
+        {
+            Assert.AreEqual(_actions.Count, 6);
+
+            TestUtils.RunMultiThreaded(() => _actions[1](Grid1.Compute()), 4, 20);
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        [Test]
+        public void TestSingleTaskType2()
+        {
+            Assert.AreEqual(_actions.Count, 6);
+
+            TestUtils.RunMultiThreaded(() => _actions[2](Grid1.Compute()), 4, 20);
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        [Test]
+        public void TestSingleTaskType3()
+        {
+            Assert.AreEqual(_actions.Count, 6);
+
+            TestUtils.RunMultiThreaded(() => _actions[3](Grid1.Compute()), 4, 20);
+        }
+        /// <summary>
+        ///
+        /// </summary>
+        [Test]
+        public void TestSingleTaskType4()
+        {
+            Assert.AreEqual(_actions.Count, 6);
+
+            TestUtils.RunMultiThreaded(() => _actions[4](Grid1.Compute()), 4, 20);
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        [Test]
+        public void TestSingleTaskType5()
+        {
+            Assert.AreEqual(_actions.Count, 6);
+
+            TestUtils.RunMultiThreaded(() => _actions[5](Grid1.Compute()), 4, 20);
+        }
+    }
+
+    /// <summary>
+    /// Test class.
+    /// </summary>
+    [Serializable]
+    public class My1ArgClosure : IComputeFunc<string, int>
+    {
+        /** <inheritDoc /> */
+        public int Invoke(string s)
+        {
+            return s.Length;
+        }
+    }
+
+    /// <summary>
+    /// Test class.
+    /// </summary>
+    [Serializable]
+    public class MyNoArgClosure : IComputeFunc<int>
+    {
+        /** */
+        private readonly string _s;
+
+        /// <summary>
+        ///
+        /// </summary>
+        /// <param name="s"></param>
+        public MyNoArgClosure(string s)
+        {
+            _s = s;
+        }
+
+        /** <inheritDoc /> */
+        public int Invoke()
+        {
+            return _s.Length;
+        }
+    }
+
+    /// <summary>
+    ///
+    /// </summary>
+    public class StringLengthEmptyTask : IComputeTask<string, int, int>
+    {
+        /** <inheritDoc /> */
+        public IDictionary<IComputeJob<int>, IClusterNode> Map(IList<IClusterNode> subgrid, string arg)
+        {
+            var res = new Dictionary<IComputeJob<int>, IClusterNode>();
+
+            var job = new StringLengthEmptyJob(arg);
+
+            IClusterNode node = subgrid[TestUtils.Random.Next(subgrid.Count)];
+
+            res.Add(job, node);
+
+            return res;
+        }
+
+        /** <inheritDoc /> */
+        public ComputeJobResultPolicy Result(IComputeJobResult<int> res, IList<IComputeJobResult<int>> rcvd)
+        {
+            return ComputeJobResultPolicy.Wait;
+        }
+
+        /** <inheritDoc /> */
+        public int Reduce(IList<IComputeJobResult<int>> results)
+        {
+            return results.Count == 0 ? 0 : results[0].Data();
+        }
+    }
+
+    /// <summary>
+    /// Test class.
+    /// </summary>
+    [Serializable]
+    public class StringLengthEmptyJob: IComputeJob<int>
+    {
+        /** */
+        private string _s;
+
+        /// <summary>
+        ///
+        /// </summary>
+        /// <param name="s"></param>
+        public StringLengthEmptyJob(string s)
+        {
+            _s = s;
+        }
+
+        /** <inheritDoc /> */
+        public int Execute()
+        {
+            return _s.Length;
+        }
+
+        /** <inheritDoc /> */
+        public void Cancel()
+        {
+            // No-op
+        }
+    }
+
+    public class MyReducer : IComputeReducer<int, int>
+    {
+        /** */
+        private int _res;
+
+        public bool Collect(int res)
+        {
+            _res += res;
+            return true;
+        }
+
+        public int Reduce()
+        {
+            return _res;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/FailoverTaskSelfTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/FailoverTaskSelfTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/FailoverTaskSelfTest.cs
new file mode 100644
index 0000000..6240bb9
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/FailoverTaskSelfTest.cs
@@ -0,0 +1,246 @@
+/*
+ * 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.Compute
+{
+    using System;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Portable;
+    using Apache.Ignite.Core.Resource;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Test for task and job adapter.
+    /// </summary>
+    public class FailoverTaskSelfTest : AbstractTaskTest
+    {
+        /** */
+        static volatile string _gridName;
+
+        /** */
+        static volatile int _cnt;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        public FailoverTaskSelfTest() : base(false) { }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="fork">Fork flag.</param>
+        protected FailoverTaskSelfTest(bool fork) : base(fork) { }
+
+        /// <summary>
+        /// Test for GridComputeJobFailoverException.
+        /// </summary>
+        [Test]
+        public void TestClosureFailoverException()
+        {
+            for (int i = 0; i < 20; i++)
+            {
+                int res = Grid1.Compute().Call(new TestClosure());
+
+                Assert.AreEqual(2, res);
+
+                Cleanup();
+            }
+        }
+
+        /// <summary>
+        /// Test for GridComputeJobFailoverException with serializable job.
+        /// </summary>
+        [Test]
+        public void TestTaskAdapterFailoverExceptionSerializable()
+        {
+            TestTaskAdapterFailoverException(true);
+        }
+
+        /// <summary>
+        /// Test for GridComputeJobFailoverException with portable job.
+        /// </summary>
+        [Test]
+        public void TestTaskAdapterFailoverExceptionPortable()
+        {
+            TestTaskAdapterFailoverException(false);
+        }
+
+        /// <summary>
+        /// Test for GridComputeJobFailoverException.
+        /// </summary>
+        private void TestTaskAdapterFailoverException(bool serializable)
+        {
+            int res = Grid1.Compute().Execute(new TestTask(),
+                new Tuple<bool, bool>(serializable, true));
+
+            Assert.AreEqual(2, res);
+
+            Cleanup();
+
+            res = Grid1.Compute().Execute(new TestTask(),
+                new Tuple<bool, bool>(serializable, false));
+
+            Assert.AreEqual(2, res);
+        }
+
+        /// <summary>
+        /// Cleanup.
+        /// </summary>
+        [TearDown]
+        public void Cleanup()
+        {
+            _cnt = 0;
+
+            _gridName = null;
+        }
+
+        /** <inheritDoc /> */
+        override protected void PortableTypeConfigurations(ICollection<PortableTypeConfiguration> portTypeCfgs)
+        {
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(TestPortableJob)));
+        }
+
+        /// <summary>
+        /// Test task.
+        /// </summary>
+        public class TestTask : ComputeTaskAdapter<Tuple<bool, bool>, int, int>
+        {
+            /** <inheritDoc /> */
+            override public IDictionary<IComputeJob<int>, IClusterNode> Map(IList<IClusterNode> subgrid, Tuple<bool, bool> arg)
+            {
+                Assert.AreEqual(3, subgrid.Count);
+
+                Tuple<bool, bool> t = arg;
+
+                bool serializable = t.Item1;
+                bool local = t.Item2;
+
+                IDictionary<IComputeJob<int>, IClusterNode> jobs = new Dictionary<IComputeJob<int>, IClusterNode>();
+
+                IComputeJob<int> job;
+
+                if (serializable)
+                    job = new TestSerializableJob();
+                else
+                    job = new TestPortableJob();
+
+                foreach (IClusterNode node in subgrid) {
+                    bool add = local ? node.IsLocal : !node.IsLocal;
+
+                    if (add)
+                    {
+                        jobs.Add(job, node);
+
+                        break;
+                    }
+                }
+
+                Assert.AreEqual(1, jobs.Count);
+
+                return jobs;
+            }
+
+            /** <inheritDoc /> */
+            override public int Reduce(IList<IComputeJobResult<int>> results)
+            {
+                Assert.AreEqual(1, results.Count);
+
+                return results[0].Data();
+            }
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        [Serializable]
+        class TestClosure : IComputeFunc<int>
+        {
+            [InstanceResource]
+            private IIgnite _grid = null;
+
+            /** <inheritDoc /> */
+            public int Invoke()
+            {
+                return FailoverJob(_grid);
+            }
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        [Serializable]
+        class TestSerializableJob : IComputeJob<int>
+        {
+            [InstanceResource]
+            private IIgnite _grid = null;
+
+            /** <inheritDoc /> */
+            public int Execute()
+            {
+                return FailoverJob(_grid);
+            }
+
+            /** <inheritDoc /> */
+            public void Cancel()
+            {
+                // No-op.
+            }
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        class TestPortableJob : IComputeJob<int>
+        {
+            [InstanceResource]
+            private IIgnite _grid = null;
+
+            /** <inheritDoc /> */
+            public int Execute()
+            {
+                return FailoverJob(_grid);
+            }
+
+            public void Cancel()
+            {
+                // No-op.
+            }
+        }
+
+        /// <summary>
+        /// Throws GridComputeJobFailoverException on first call.
+        /// </summary>
+        private static int FailoverJob(IIgnite grid)
+        {
+            Assert.NotNull(grid);
+
+            _cnt++;
+
+            if (_gridName == null)
+            {
+                _gridName = grid.Name;
+
+                throw new ComputeJobFailoverException("Test error.");
+            }
+            Assert.AreNotEqual(_gridName, grid.Name);
+
+            return _cnt;
+        }
+    }
+}


[06/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/LifecycleTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/LifecycleTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/LifecycleTest.cs
new file mode 100644
index 0000000..3c38ef9
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/LifecycleTest.cs
@@ -0,0 +1,288 @@
+/*
+ * 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.Collections.Generic;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.Lifecycle;
+    using Apache.Ignite.Core.Resource;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Lifecycle beans test.
+    /// </summary>
+    public class LifecycleTest
+    {
+        /** Configuration: without Java beans. */
+        private const string CfgNoBeans = "config//lifecycle//lifecycle-no-beans.xml";
+
+        /** Configuration: with Java beans. */
+        private const string CfgBeans = "config//lifecycle//lifecycle-beans.xml";
+
+        /** Whether to throw an error on lifecycle event. */
+        internal static bool ThrowErr;
+
+        /** Events: before start. */
+        internal static IList<Event> BeforeStartEvts;
+
+        /** Events: after start. */
+        internal static IList<Event> AfterStartEvts;
+
+        /** Events: before stop. */
+        internal static IList<Event> BeforeStopEvts;
+
+        /** Events: after stop. */
+        internal static IList<Event> AfterStopEvts;
+
+        /// <summary>
+        /// Set up routine.
+        /// </summary>
+        [SetUp]
+        public void SetUp()
+        {
+            ThrowErr = false;
+
+            BeforeStartEvts = new List<Event>();
+            AfterStartEvts = new List<Event>();
+            BeforeStopEvts = new List<Event>();
+            AfterStopEvts = new List<Event>();
+        }
+
+        /// <summary>
+        /// Tear down routine.
+        /// </summary>
+        [TearDown]
+        public void TearDown()
+        {
+            Ignition.StopAll(true);
+        }
+        
+        /// <summary>
+        /// Test without Java beans.
+        /// </summary>
+        [Test]
+        public void TestWithoutBeans()
+        {
+            // 1. Test start events.
+            IIgnite grid = Start(CfgNoBeans);
+
+            Assert.AreEqual(2, BeforeStartEvts.Count);
+            CheckEvent(BeforeStartEvts[0], null, null, 0, null);
+            CheckEvent(BeforeStartEvts[1], null, null, 0, null);
+
+            Assert.AreEqual(2, AfterStartEvts.Count);
+            CheckEvent(AfterStartEvts[0], grid, grid, 0, null);
+            CheckEvent(AfterStartEvts[1], grid, grid, 0, null);
+
+            // 2. Test stop events.
+            Ignition.Stop(grid.Name, false);
+
+            Assert.AreEqual(2, BeforeStartEvts.Count);
+            Assert.AreEqual(2, AfterStartEvts.Count);
+
+            Assert.AreEqual(2, BeforeStopEvts.Count);
+            CheckEvent(BeforeStopEvts[0], grid, grid, 0, null);
+            CheckEvent(BeforeStopEvts[1], grid, grid, 0, null);
+
+            Assert.AreEqual(2, AfterStopEvts.Count);
+            CheckEvent(AfterStopEvts[0], grid, grid, 0, null);
+            CheckEvent(AfterStopEvts[1], grid, grid, 0, null);
+        }
+
+        /// <summary>
+        /// Test with Java beans.
+        /// </summary>
+        [Test]
+        public void TestWithBeans()
+        {
+            // 1. Test .Net start events.
+            IIgnite grid = Start(CfgBeans);
+
+            Assert.AreEqual(4, BeforeStartEvts.Count);
+            CheckEvent(BeforeStartEvts[0], null, null, 0, null);
+            CheckEvent(BeforeStartEvts[1], null, null, 1, "1");
+            CheckEvent(BeforeStartEvts[2], null, null, 0, null);
+            CheckEvent(BeforeStartEvts[3], null, null, 0, null);
+
+            Assert.AreEqual(4, AfterStartEvts.Count);
+            CheckEvent(AfterStartEvts[0], grid, grid, 0, null);
+            CheckEvent(AfterStartEvts[1], grid, grid, 1, "1");
+            CheckEvent(AfterStartEvts[2], grid, grid, 0, null);
+            CheckEvent(AfterStartEvts[3], grid, grid, 0, null);
+
+            // 2. Test Java start events.
+            IList<int> res = grid.Compute().ExecuteJavaTask<IList<int>>(
+                "org.apache.ignite.platform.lifecycle.PlatformJavaLifecycleTask", null);
+
+            Assert.AreEqual(2, res.Count);
+            Assert.AreEqual(3, res[0]);
+            Assert.AreEqual(3, res[1]);
+
+            // 3. Test .Net stop events.
+            Ignition.Stop(grid.Name, false);
+
+            Assert.AreEqual(4, BeforeStartEvts.Count);
+            Assert.AreEqual(4, AfterStartEvts.Count);
+
+            Assert.AreEqual(4, BeforeStopEvts.Count);
+            CheckEvent(BeforeStopEvts[0], grid, grid, 0, null);
+            CheckEvent(BeforeStopEvts[1], grid, grid, 1, "1");
+            CheckEvent(BeforeStopEvts[2], grid, grid, 0, null);
+            CheckEvent(BeforeStopEvts[3], grid, grid, 0, null);
+
+            Assert.AreEqual(4, AfterStopEvts.Count);
+            CheckEvent(AfterStopEvts[0], grid, grid, 0, null);
+            CheckEvent(AfterStopEvts[1], grid, grid, 1, "1");
+            CheckEvent(AfterStopEvts[2], grid, grid, 0, null);
+            CheckEvent(AfterStopEvts[3], grid, grid, 0, null);
+        }
+
+        /// <summary>
+        /// Test behavior when error is thrown from lifecycle beans.
+        /// </summary>
+        [Test]
+        public void TestError()
+        {
+            ThrowErr = true;
+
+            try
+            {
+                Start(CfgNoBeans);
+
+                Assert.Fail("Should not reach this place.");
+            }
+            catch (Exception e)
+            {
+                Assert.AreEqual(typeof(IgniteException), e.GetType());
+            }
+        }
+
+        /// <summary>
+        /// Start grid.
+        /// </summary>
+        /// <param name="cfgPath">Spring configuration path.</param>
+        /// <returns>Grid.</returns>
+        private static IIgnite Start(string cfgPath)
+        {
+            TestUtils.JvmDebug = true;
+
+            IgniteConfiguration cfg = new IgniteConfiguration();
+
+            cfg.JvmClasspath = TestUtils.CreateTestClasspath();
+            cfg.JvmOptions = TestUtils.TestJavaOptions();
+            cfg.SpringConfigUrl = cfgPath;
+
+            cfg.LifecycleBeans = new List<ILifecycleBean> { new Bean(), new Bean() };
+
+            return Ignition.Start(cfg);
+        }
+
+        /// <summary>
+        /// Check event.
+        /// </summary>
+        /// <param name="evt">Event.</param>
+        /// <param name="expGrid1">Expected grid 1.</param>
+        /// <param name="expGrid2">Expected grid 2.</param>
+        /// <param name="expProp1">Expected property 1.</param>
+        /// <param name="expProp2">Expected property 2.</param>
+        private static void CheckEvent(Event evt, IIgnite expGrid1, IIgnite expGrid2, int expProp1, string expProp2)
+        {
+            if (evt.Grid1 != null && evt.Grid1 is IgniteProxy)
+                evt.Grid1 = (evt.Grid1 as IgniteProxy).Target;
+
+            if (evt.Grid2 != null && evt.Grid2 is IgniteProxy)
+                evt.Grid2 = (evt.Grid2 as IgniteProxy).Target;
+
+            Assert.AreEqual(expGrid1, evt.Grid1);
+            Assert.AreEqual(expGrid2, evt.Grid2);
+            Assert.AreEqual(expProp1, evt.Prop1);
+            Assert.AreEqual(expProp2, evt.Prop2);
+        }
+    }
+
+    public abstract class AbstractBean
+    {
+        [InstanceResource]
+        public IIgnite Grid1;
+
+        public int Property1
+        {
+            get;
+            set;
+        }
+    }
+
+    public class Bean : AbstractBean, ILifecycleBean
+    {
+        [InstanceResource]
+        public IIgnite Grid2;
+
+        public string Property2
+        {
+            get;
+            set;
+        }
+
+        /** <inheritDoc /> */
+        public void OnLifecycleEvent(LifecycleEventType evtType)
+        {
+            if (LifecycleTest.ThrowErr)
+                throw new Exception("Lifecycle exception.");
+
+            Event evt = new Event();
+
+            evt.Grid1 = Grid1;
+            evt.Grid2 = Grid2;
+            evt.Prop1 = Property1;
+            evt.Prop2 = Property2;
+
+            switch (evtType)
+            {
+                case LifecycleEventType.BeforeNodeStart:
+                    LifecycleTest.BeforeStartEvts.Add(evt);
+
+                    break;
+
+                case LifecycleEventType.AfterNodeStart:
+                    LifecycleTest.AfterStartEvts.Add(evt);
+
+                    break;
+
+                case LifecycleEventType.BeforeNodeStop:
+                    LifecycleTest.BeforeStopEvts.Add(evt);
+
+                    break;
+
+                case LifecycleEventType.AfterNodeStop:
+                    LifecycleTest.AfterStopEvts.Add(evt);
+
+                    break;
+            }
+        }
+    }
+
+    public class Event
+    {
+        public IIgnite Grid1;
+        public IIgnite Grid2;
+        public int Prop1;
+        public string Prop2;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/LoadDllTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/LoadDllTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/LoadDllTest.cs
new file mode 100644
index 0000000..af9387c
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/LoadDllTest.cs
@@ -0,0 +1,243 @@
+/*
+ * 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.CodeDom.Compiler;
+    using System.Collections.Generic;
+    using System.IO;
+    using System.Linq;
+    using Apache.Ignite.Core.Common;
+    using Microsoft.CSharp;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Dll loading test.
+    /// </summary>
+    public class LoadDllTest
+    {
+        /// <summary>
+        /// 
+        /// </summary>
+        [SetUp]
+        public void SetUp()
+        {
+            TestUtils.KillProcesses();
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        [TearDown]
+        public void TearDown()
+        {
+            Ignition.StopAll(true);
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        [Test]
+        public void TestLoadFromGac()
+        {
+            Assert.False(IsLoaded("System.Data.Linq"));
+
+            var cfg = new IgniteConfiguration
+            {
+                SpringConfigUrl = "config\\start-test-grid3.xml",
+                Assemblies =
+                    new List<string>
+                    {
+                        "System.Data.Linq,Culture=neutral,Version=1.0.0.0,PublicKeyToken=b77a5c561934e089"
+                    },
+                JvmClasspath = TestUtils.CreateTestClasspath()
+            };
+
+
+            var grid = Ignition.Start(cfg);
+
+            Assert.IsNotNull(grid);
+
+            Assert.True(IsLoaded("System.Data.Linq"));
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        [Test]
+        public void TestLoadFromCurrentDir()
+        {
+            Assert.False(IsLoaded("testDll"));
+
+            GenerateDll("testDll.dll");
+
+            var cfg = new IgniteConfiguration
+            {
+                SpringConfigUrl = "config\\start-test-grid3.xml",
+                Assemblies = new List<string> {"testDll.dll"},
+                JvmClasspath = TestUtils.CreateTestClasspath()
+            };
+
+            var grid = Ignition.Start(cfg);
+
+            Assert.IsNotNull(grid);
+
+            Assert.True(IsLoaded("testDll"));
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        [Test]
+        public void TestLoadAllDllInDir()
+        {
+            var dirInfo = Directory.CreateDirectory(Path.GetTempPath() + "/testDlls");
+            
+            Assert.False(IsLoaded("dllFromDir1"));
+            Assert.False(IsLoaded("dllFromDir2"));
+
+            GenerateDll(dirInfo.FullName + "/dllFromDir1.dll");
+            GenerateDll(dirInfo.FullName + "/dllFromDir2.dll");
+            File.WriteAllText(dirInfo.FullName + "/notADll.txt", "notADll");
+
+            var cfg = new IgniteConfiguration
+            {
+                SpringConfigUrl = "config\\start-test-grid3.xml",
+                Assemblies = new List<string> {dirInfo.FullName},
+                JvmClasspath = TestUtils.CreateTestClasspath()
+            };
+
+            var grid = Ignition.Start(cfg);
+
+            Assert.IsNotNull(grid);
+
+            Assert.True(IsLoaded("dllFromDir1"));
+            Assert.True(IsLoaded("dllFromDir2"));
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        [Test]
+        public void TestLoadFromCurrentDirByName()
+        {
+            Assert.False(IsLoaded("testDllByName"));
+
+            GenerateDll("testDllByName.dll");
+
+            var cfg = new IgniteConfiguration
+            {
+                SpringConfigUrl = "config\\start-test-grid3.xml",
+                Assemblies = new List<string> {"testDllByName"},
+                JvmClasspath = TestUtils.CreateTestClasspath()
+            };
+
+            var grid = Ignition.Start(cfg);
+
+            Assert.IsNotNull(grid);
+
+            Assert.True(IsLoaded("testDllByName"));
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        [Test]
+        public void TestLoadByAbsoluteUri()
+        {
+            var dllPath = Path.GetTempPath() + "/tempDll.dll";
+            Assert.False(IsLoaded("tempDll"));
+
+            GenerateDll(dllPath);
+
+            var cfg = new IgniteConfiguration
+            {
+                SpringConfigUrl = "config\\start-test-grid3.xml",
+                Assemblies = new List<string> {dllPath},
+                JvmClasspath = TestUtils.CreateTestClasspath()
+            };
+
+            var grid = Ignition.Start(cfg);
+
+            Assert.IsNotNull(grid);
+
+            Assert.True(IsLoaded("tempDll"));
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        [Test]
+        public void TestLoadUnexistingLibrary()
+        {
+            var cfg = new IgniteConfiguration
+            {
+                SpringConfigUrl = "config\\start-test-grid3.xml",
+                Assemblies = new List<string> {"unexistingAssembly.820482.dll"},
+                JvmClasspath = TestUtils.CreateTestClasspath()
+            };
+
+            try
+            {
+                Ignition.Start(cfg);
+
+                Assert.Fail("Grid has been started with broken configuration.");
+            }
+            catch (IgniteException)
+            {
+
+            }
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        /// <param name="outputPath"></param>
+        private void GenerateDll(string outputPath)
+        {
+            var codeProvider = new CSharpCodeProvider();
+
+#pragma warning disable 0618
+
+            var icc = codeProvider.CreateCompiler();
+
+#pragma warning restore 0618
+
+            var parameters = new CompilerParameters
+            {
+                GenerateExecutable = false,
+                OutputAssembly = outputPath
+            };
+
+            var src = "namespace GridGain.Client.Test { public class Foo {}}";
+
+            var results = icc.CompileAssemblyFromSource(parameters, src);
+
+            Assert.False(results.Errors.HasErrors);
+        }
+
+        /// <summary>
+        /// Determines whether the specified assembly is loaded.
+        /// </summary>
+        /// <param name="asmName">Name of the assembly.</param>
+        private static bool IsLoaded(string asmName)
+        {
+            return AppDomain.CurrentDomain.GetAssemblies().Any(a => a.GetName().Name == asmName);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/MarshallerTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/MarshallerTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/MarshallerTest.cs
new file mode 100644
index 0000000..d3af288
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/MarshallerTest.cs
@@ -0,0 +1,71 @@
+/*
+ * 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 Apache.Ignite.Core.Common;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Test marshaller initialization.
+    /// </summary>
+    public class MarshallerTest
+    {
+        /// <summary>
+        /// Tests the default marhsaller.
+        /// By default, portable marshaller is used.
+        /// </summary>
+        [Test]
+        public void TestDefaultMarhsaller()
+        {
+            using (var grid = Ignition.Start("config\\marshaller-default.xml"))
+            {
+                var cache = grid.GetOrCreateCache<int, int>(null);
+
+                cache.Put(1, 1);
+
+                Assert.AreEqual(1, cache.Get(1));
+            }
+        }
+
+        /// <summary>
+        /// Tests the portable marhsaller.
+        /// PortableMarshaller can be specified explicitly in config.
+        /// </summary>
+        [Test]
+        public void TestPortableMarhsaller()
+        {
+            using (var grid = Ignition.Start("config\\marshaller-portable.xml"))
+            {
+                var cache = grid.GetOrCreateCache<int, int>(null);
+
+                cache.Put(1, 1);
+
+                Assert.AreEqual(1, cache.Get(1));
+            }
+        }
+
+        /// <summary>
+        /// Tests the invalid marshaller.
+        /// </summary>
+        [Test]
+        public void TestInvalidMarshaller()
+        {
+            Assert.Throws<IgniteException>(() => Ignition.Start("config\\marshaller-invalid.xml"));
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs
new file mode 100644
index 0000000..abb8e2f
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs
@@ -0,0 +1,646 @@
+/*
+ * 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.Collections.Concurrent;
+    using System.Collections.Generic;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Linq;
+    using System.Threading;
+    using System.Threading.Tasks;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Messaging;
+    using Apache.Ignite.Core.Resource;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// <see cref="IMessaging"/> tests.
+    /// </summary>
+    public class MessagingTest
+    {
+        /** */
+        private IIgnite _grid1;
+
+        /** */
+        private IIgnite _grid2;
+
+        /** */
+        private IIgnite _grid3;
+
+        /** */
+        public static int MessageId;
+
+        /// <summary>
+        /// Executes before each test.
+        /// </summary>
+        [SetUp]
+        public void SetUp()
+        {
+            _grid1 = Ignition.Start(Configuration("config\\compute\\compute-grid1.xml"));
+            _grid2 = Ignition.Start(Configuration("config\\compute\\compute-grid2.xml"));
+            _grid3 = Ignition.Start(Configuration("config\\compute\\compute-grid3.xml"));
+        }
+
+        /// <summary>
+        /// Executes after each test.
+        /// </summary>
+        [TearDown]
+        public virtual void TearDown()
+        {
+            try
+            {
+                TestUtils.AssertHandleRegistryIsEmpty(1000, _grid1, _grid2, _grid3);
+
+                MessagingTestHelper.AssertFailures();
+            }
+            finally 
+            {
+                // Stop all grids between tests to drop any hanging messages
+                Ignition.StopAll(true);
+            }
+        }
+
+        /// <summary>
+        /// Tests LocalListen.
+        /// </summary>
+        [Test]
+        public void TestLocalListen()
+        {
+            TestLocalListen(null);
+            TestLocalListen("string topic");
+            TestLocalListen(NextId());
+        }
+
+        /// <summary>
+        /// Tests LocalListen.
+        /// </summary>
+        [SuppressMessage("ReSharper", "AccessToModifiedClosure")]
+        public void TestLocalListen(object topic)
+        {
+            var messaging = _grid1.Message();
+            var listener = MessagingTestHelper.GetListener();
+            messaging.LocalListen(listener, topic);
+
+            // Test sending
+            CheckSend(topic);
+            CheckSend(topic, _grid2);
+            CheckSend(topic, _grid3);
+
+            // Test different topic
+            CheckNoMessage(NextId());
+            CheckNoMessage(NextId(), _grid2);
+
+            // Test multiple subscriptions for the same filter
+            messaging.LocalListen(listener, topic);
+            messaging.LocalListen(listener, topic);
+            CheckSend(topic, repeatMultiplier: 3); // expect all messages repeated 3 times
+
+            messaging.StopLocalListen(listener, topic);
+            CheckSend(topic, repeatMultiplier: 2); // expect all messages repeated 2 times
+
+            messaging.StopLocalListen(listener, topic);
+            CheckSend(topic); // back to 1 listener
+
+            // Test message type mismatch
+            var ex = Assert.Throws<IgniteException>(() => messaging.Send(1.1, topic));
+            Assert.AreEqual("Unable to cast object of type 'System.Double' to type 'System.String'.", ex.Message);
+
+            // Test end listen
+            MessagingTestHelper.ListenResult = false;
+            CheckSend(topic, single: true); // we'll receive one more and then unsubscribe because of delegate result.
+            CheckNoMessage(topic);
+
+            // Start again
+            MessagingTestHelper.ListenResult = true;
+            messaging.LocalListen(listener, topic);
+            CheckSend(topic);
+
+            // Stop
+            messaging.StopLocalListen(listener, topic);
+            CheckNoMessage(topic);
+        }
+
+        /// <summary>
+        /// Tests LocalListen with projection.
+        /// </summary>
+        [Test]
+        public void TestLocalListenProjection()
+        {
+            TestLocalListenProjection(null);
+            TestLocalListenProjection("prj");
+            TestLocalListenProjection(NextId());
+        }
+
+        /// <summary>
+        /// Tests LocalListen with projection.
+        /// </summary>
+        private void TestLocalListenProjection(object topic)
+        {
+            var grid3GotMessage = false;
+
+            var grid3Listener = new MessageFilter<string>((id, x) =>
+            {
+                grid3GotMessage = true;
+                return true;
+            });
+
+            _grid3.Message().LocalListen(grid3Listener, topic);
+
+            var clusterMessaging = _grid1.Cluster.ForNodes(_grid1.Cluster.LocalNode, _grid2.Cluster.LocalNode).Message();
+            var clusterListener = MessagingTestHelper.GetListener();
+            clusterMessaging.LocalListen(clusterListener, topic);
+
+            CheckSend(msg: clusterMessaging, topic: topic);
+            Assert.IsFalse(grid3GotMessage, "Grid3 should not get messages");
+
+            CheckSend(grid: _grid2, msg: clusterMessaging, topic: topic);
+            Assert.IsFalse(grid3GotMessage, "Grid3 should not get messages");
+
+            clusterMessaging.StopLocalListen(clusterListener, topic);
+            _grid3.Message().StopLocalListen(grid3Listener, topic);
+        }
+
+        /// <summary>
+        /// Tests LocalListen in multithreaded mode.
+        /// </summary>
+        [Test]
+        [SuppressMessage("ReSharper", "AccessToModifiedClosure")]
+        [Category(TestUtils.CategoryIntensive)]
+        public void TestLocalListenMultithreaded()
+        {
+            const int threadCnt = 20;
+            const int runSeconds = 20;
+
+            var messaging = _grid1.Message();
+
+            var senders = Task.Factory.StartNew(() => TestUtils.RunMultiThreaded(() =>
+            {
+                messaging.Send((object) NextMessage());
+                Thread.Sleep(50);
+            }, threadCnt, runSeconds));
+
+
+            var sharedReceived = 0;
+
+            var sharedListener = new MessageFilter<string>((id, x) =>
+            {
+                Interlocked.Increment(ref sharedReceived);
+                Thread.MemoryBarrier();
+                return true;
+            });
+
+            TestUtils.RunMultiThreaded(() =>
+            {
+                // Check that listen/stop work concurrently
+                messaging.LocalListen(sharedListener);
+
+                for (int i = 0; i < 100; i++)
+                {
+                    messaging.LocalListen(sharedListener);
+                    messaging.StopLocalListen(sharedListener);
+                }
+
+                var localReceived = 0;
+                var stopLocal = 0;
+
+                var localListener = new MessageFilter<string>((id, x) =>
+                {
+                    Interlocked.Increment(ref localReceived);
+                    Thread.MemoryBarrier();
+                    return Thread.VolatileRead(ref stopLocal) == 0;
+                });
+
+                messaging.LocalListen(localListener);
+
+                Thread.Sleep(100);
+
+                Thread.VolatileWrite(ref stopLocal, 1);
+
+                Thread.Sleep(1000);
+
+                var result = Thread.VolatileRead(ref localReceived);
+
+                Thread.Sleep(100);
+
+                // Check that unsubscription worked properly
+                Assert.AreEqual(result, Thread.VolatileRead(ref localReceived));
+
+                messaging.StopLocalListen(sharedListener);
+
+            }, threadCnt, runSeconds);
+
+            senders.Wait();
+
+            Thread.Sleep(100);
+
+            var sharedResult = Thread.VolatileRead(ref sharedReceived);
+
+            messaging.Send((object)NextMessage());
+
+            Thread.Sleep(MessagingTestHelper.MessageTimeout);
+
+            // Check that unsubscription worked properly
+            Assert.AreEqual(sharedResult, Thread.VolatileRead(ref sharedReceived));
+        }
+
+        /// <summary>
+        /// Tests RemoteListen.
+        /// </summary>
+        [Test]
+        public void TestRemoteListen()
+        {
+            TestRemoteListen(null);
+            TestRemoteListen("string topic");
+            TestRemoteListen(NextId());
+        }
+
+        /// <summary>
+        /// Tests RemoteListen with async mode enabled.
+        /// </summary>
+        [Test]
+        public void TestRemoteListenAsync()
+        {
+            TestRemoteListen(null, true);
+            TestRemoteListen("string topic", true);
+            TestRemoteListen(NextId(), true);
+        }
+
+        /// <summary>
+        /// Tests RemoteListen.
+        /// </summary>
+        public void TestRemoteListen(object topic, bool async = false)
+        {
+            var messaging = async ? _grid1.Message().WithAsync() : _grid1.Message();
+
+            var listener = MessagingTestHelper.GetListener();
+            var listenId = messaging.RemoteListen(listener, topic);
+
+            if (async)
+                listenId = messaging.GetFuture<Guid>().Get();
+
+            // Test sending
+            CheckSend(topic, msg: messaging, remoteListen: true);
+
+            // Test different topic
+            CheckNoMessage(NextId());
+
+            // Test multiple subscriptions for the same filter
+            var listenId2 = messaging.RemoteListen(listener, topic);
+
+            if (async)
+                listenId2 = messaging.GetFuture<Guid>().Get();
+
+            CheckSend(topic, msg: messaging, remoteListen: true, repeatMultiplier: 2); // expect twice the messages
+
+            messaging.StopRemoteListen(listenId2);
+
+            if (async)
+                messaging.GetFuture().Get();
+
+            CheckSend(topic, msg: messaging, remoteListen: true); // back to normal after unsubscription
+
+            // Test message type mismatch
+            var ex = Assert.Throws<IgniteException>(() => messaging.Send(1.1, topic));
+            Assert.AreEqual("Unable to cast object of type 'System.Double' to type 'System.String'.", ex.Message);
+
+            // Test end listen
+            messaging.StopRemoteListen(listenId);
+
+            if (async)
+                messaging.GetFuture().Get();
+
+            CheckNoMessage(topic);
+        }
+
+        /// <summary>
+        /// Tests RemoteListen with a projection.
+        /// </summary>
+        [Test]
+        public void TestRemoteListenProjection()
+        {
+            TestRemoteListenProjection(null);
+            TestRemoteListenProjection("string topic");
+            TestRemoteListenProjection(NextId());
+        }
+
+        /// <summary>
+        /// Tests RemoteListen with a projection.
+        /// </summary>
+        private void TestRemoteListenProjection(object topic)
+        {
+            var clusterMessaging = _grid1.Cluster.ForNodes(_grid1.Cluster.LocalNode, _grid2.Cluster.LocalNode).Message();
+            var clusterListener = MessagingTestHelper.GetListener();
+            var listenId = clusterMessaging.RemoteListen(clusterListener, topic);
+
+            CheckSend(msg: clusterMessaging, topic: topic, remoteListen: true);
+
+            clusterMessaging.StopRemoteListen(listenId);
+
+            CheckNoMessage(topic);
+        }
+
+        /// <summary>
+        /// Tests LocalListen in multithreaded mode.
+        /// </summary>
+        [Test]
+        [Category(TestUtils.CategoryIntensive)]
+        public void TestRemoteListenMultithreaded()
+        {
+            const int threadCnt = 20;
+            const int runSeconds = 20;
+
+            var messaging = _grid1.Message();
+
+            var senders = Task.Factory.StartNew(() => TestUtils.RunMultiThreaded(() =>
+            {
+                MessagingTestHelper.ClearReceived(int.MaxValue);
+                messaging.Send((object) NextMessage());
+                Thread.Sleep(50);
+            }, threadCnt, runSeconds));
+
+
+            var sharedListener = MessagingTestHelper.GetListener();
+
+            for (int i = 0; i < 100; i++)
+                messaging.RemoteListen(sharedListener);  // add some listeners to be stopped by filter result
+
+            TestUtils.RunMultiThreaded(() =>
+            {
+                // Check that listen/stop work concurrently
+                messaging.StopRemoteListen(messaging.RemoteListen(sharedListener));
+
+            }, threadCnt, runSeconds);
+
+            MessagingTestHelper.ListenResult = false;
+
+            messaging.Send((object) NextMessage()); // send a message to make filters return false
+
+            Thread.Sleep(MessagingTestHelper.MessageTimeout); // wait for all to unsubscribe
+
+            MessagingTestHelper.ListenResult = true;
+
+            senders.Wait(); // wait for senders to stop
+
+            var sharedResult = MessagingTestHelper.ReceivedMessages.Count;
+
+            messaging.Send((object) NextMessage());
+
+            Thread.Sleep(MessagingTestHelper.MessageTimeout);
+
+            // Check that unsubscription worked properly
+            Assert.AreEqual(sharedResult, MessagingTestHelper.ReceivedMessages.Count);
+            
+        }
+
+        /// <summary>
+        /// Sends messages in various ways and verefies correct receival.
+        /// </summary>
+        /// <param name="topic">Topic.</param>
+        /// <param name="grid">The grid to use.</param>
+        /// <param name="msg">Messaging to use.</param>
+        /// <param name="remoteListen">Whether to expect remote listeners.</param>
+        /// <param name="single">When true, only check one message.</param>
+        /// <param name="repeatMultiplier">Expected message count multiplier.</param>
+        private void CheckSend(object topic = null, IIgnite grid = null,
+            IMessaging msg = null, bool remoteListen = false, bool single = false, int repeatMultiplier = 1)
+        {
+            IClusterGroup cluster;
+
+            if (msg != null)
+                cluster = msg.ClusterGroup;
+            else
+            {
+                grid = grid ?? _grid1;
+                msg = grid.Message();
+                cluster = grid.Cluster.ForLocal();
+            }
+
+            // Messages will repeat due to multiple nodes listening
+            var expectedRepeat = repeatMultiplier * (remoteListen ? cluster.Nodes().Count : 1);
+
+            var messages = Enumerable.Range(1, 10).Select(x => NextMessage()).OrderBy(x => x).ToList();
+
+            // Single message
+            MessagingTestHelper.ClearReceived(expectedRepeat);
+            msg.Send((object) messages[0], topic);
+            MessagingTestHelper.VerifyReceive(cluster, messages.Take(1), m => m.ToList(), expectedRepeat);
+
+            if (single)
+                return;
+
+            // Multiple messages (receive order is undefined)
+            MessagingTestHelper.ClearReceived(messages.Count * expectedRepeat);
+            msg.Send(messages, topic);
+            MessagingTestHelper.VerifyReceive(cluster, messages, m => m.OrderBy(x => x), expectedRepeat);
+
+            // Multiple messages, ordered
+            MessagingTestHelper.ClearReceived(messages.Count * expectedRepeat);
+            messages.ForEach(x => msg.SendOrdered(x, topic, MessagingTestHelper.MessageTimeout));
+
+            if (remoteListen) // in remote scenario messages get mixed up due to different timing on different nodes
+                MessagingTestHelper.VerifyReceive(cluster, messages, m => m.OrderBy(x => x), expectedRepeat);
+            else
+                MessagingTestHelper.VerifyReceive(cluster, messages, m => m.Reverse(), expectedRepeat);
+        }
+
+        /// <summary>
+        /// Checks that no message has arrived.
+        /// </summary>
+        private void CheckNoMessage(object topic, IIgnite grid = null)
+        {
+            // this will result in an exception in case of a message
+            MessagingTestHelper.ClearReceived(0);
+
+            (grid ?? _grid1).Message().Send(NextMessage(), topic);
+
+            Thread.Sleep(MessagingTestHelper.MessageTimeout);
+
+            MessagingTestHelper.AssertFailures();
+        }
+
+        /// <summary>
+        /// Gets the Ignite configuration.
+        /// </summary>
+        private static IgniteConfiguration Configuration(string springConfigUrl)
+        {
+            return new IgniteConfiguration
+            {
+                SpringConfigUrl = springConfigUrl,
+                JvmClasspath = TestUtils.CreateTestClasspath(),
+                JvmOptions = TestUtils.TestJavaOptions()
+            };
+        }
+
+        /// <summary>
+        /// Generates next message with sequential ID and current test name.
+        /// </summary>
+        private static string NextMessage()
+        {
+            var id = NextId();
+            return id + "_" + TestContext.CurrentContext.Test.Name;
+        }
+
+        /// <summary>
+        /// Generates next sequential ID.
+        /// </summary>
+        private static int NextId()
+        {
+            return Interlocked.Increment(ref MessageId);
+        }
+    }
+
+    /// <summary>
+    /// Messaging test helper class.
+    /// </summary>
+    [Serializable]
+    public static class MessagingTestHelper
+    {
+        /** */
+        public static readonly ConcurrentStack<string> ReceivedMessages = new ConcurrentStack<string>();
+        
+        /** */
+        public static readonly ConcurrentStack<string> Failures = new ConcurrentStack<string>();
+
+        /** */
+        public static readonly CountdownEvent ReceivedEvent = new CountdownEvent(0);
+
+        /** */
+        public static readonly ConcurrentStack<Guid> LastNodeIds = new ConcurrentStack<Guid>();
+
+        /** */
+        public static volatile bool ListenResult = true;
+
+        /** */
+        public static readonly TimeSpan MessageTimeout = TimeSpan.FromMilliseconds(700);
+
+        /// <summary>
+        /// Clears received message information.
+        /// </summary>
+        /// <param name="expectedCount">The expected count of messages to be received.</param>
+        public static void ClearReceived(int expectedCount)
+        {
+            ReceivedMessages.Clear();
+            ReceivedEvent.Reset(expectedCount);
+            LastNodeIds.Clear();
+        }
+
+        /// <summary>
+        /// Verifies received messages against expected messages.
+        /// </summary>
+        /// <param name="cluster">Cluster.</param>
+        /// <param name="expectedMessages">Expected messages.</param>
+        /// <param name="resultFunc">Result transform function.</param>
+        /// <param name="expectedRepeat">Expected repeat count.</param>
+        public static void VerifyReceive(IClusterGroup cluster, IEnumerable<string> expectedMessages,
+            Func<IEnumerable<string>, IEnumerable<string>> resultFunc, int expectedRepeat)
+        {
+            // check if expected message count has been received; Wait returns false if there were none.
+            Assert.IsTrue(ReceivedEvent.Wait(MessageTimeout));
+
+            expectedMessages = expectedMessages.SelectMany(x => Enumerable.Repeat(x, expectedRepeat));
+
+            Assert.AreEqual(expectedMessages, resultFunc(ReceivedMessages));
+
+            // check that all messages came from local node.
+            var localNodeId = cluster.Ignite.Cluster.LocalNode.Id;
+            Assert.AreEqual(localNodeId, LastNodeIds.Distinct().Single());
+            
+            AssertFailures();
+        }
+
+        /// <summary>
+        /// Gets the message listener.
+        /// </summary>
+        /// <returns>New instance of message listener.</returns>
+        public static IMessageFilter<string> GetListener()
+        {
+            return new MessageFilter<string>(Listen);
+        }
+
+        /// <summary>
+        /// Combines accumulated failures and throws an assertion, if there are any.
+        /// Clears accumulated failures.
+        /// </summary>
+        public static void AssertFailures()
+        {
+            if (Failures.Any())
+                Assert.Fail(Failures.Reverse().Aggregate((x, y) => string.Format("{0}\n{1}", x, y)));
+
+            Failures.Clear();
+        }
+
+        /// <summary>
+        /// Listen method.
+        /// </summary>
+        /// <param name="id">Originating node ID.</param>
+        /// <param name="msg">Message.</param>
+        private static bool Listen(Guid id, string msg)
+        {
+            try
+            {
+                LastNodeIds.Push(id);
+                ReceivedMessages.Push(msg);
+
+                ReceivedEvent.Signal();
+
+                return ListenResult;
+            }
+            catch (Exception ex)
+            {
+                // When executed on remote nodes, these exceptions will not go to sender, 
+                // so we have to accumulate them.
+                Failures.Push(string.Format("Exception in Listen (msg: {0}, id: {1}): {2}", msg, id, ex));
+                throw;
+            }
+        }
+    }
+
+    /// <summary>
+    /// Test message filter.
+    /// </summary>
+    [Serializable]
+    public class MessageFilter<T> : IMessageFilter<T>
+    {
+        /** */
+        private readonly Func<Guid, T, bool> _invoke;
+
+        #pragma warning disable 649
+        /** Grid. */
+        [InstanceResource]
+        private IIgnite _grid;
+        #pragma warning restore 649
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="MessageFilter{T}"/> class.
+        /// </summary>
+        /// <param name="invoke">The invoke delegate.</param>
+        public MessageFilter(Func<Guid, T, bool> invoke)
+        {
+            _invoke = invoke;
+        }
+
+        /** <inheritdoc /> */
+        public bool Invoke(Guid nodeId, T message)
+        {
+            Assert.IsNotNull(_grid);
+            return _invoke(nodeId, message);
+        }
+    }
+}


[44/45] ignite git commit: Merge remote-tracking branch 'origin/ignite-843' into ignite-843

Posted by ak...@apache.org.
Merge remote-tracking branch 'origin/ignite-843' into ignite-843


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

Branch: refs/heads/ignite-843
Commit: 5d3b12909b708be00d477a99043c8a43522d3fea
Parents: b3e1b7d 2b8acba
Author: AKuznetsov <ak...@gridgain.com>
Authored: Sat Sep 5 00:51:18 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Sat Sep 5 00:51:18 2015 +0700

----------------------------------------------------------------------
 .../main/js/controllers/caches-controller.js    | 52 ++++++----------
 .../main/js/controllers/clusters-controller.js  | 64 ++++++--------------
 .../src/main/js/controllers/common-module.js    | 37 +++++------
 .../main/js/controllers/metadata-controller.js  | 32 ++++------
 4 files changed, 66 insertions(+), 119 deletions(-)
----------------------------------------------------------------------



[14/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
new file mode 100644
index 0000000..24d2b73
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
@@ -0,0 +1,3321 @@
+/*
+ * 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.Cache
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Linq;
+    using System.Runtime.Serialization;
+    using System.Text;
+    using System.Threading;
+    using System.Threading.Tasks;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cache.Expiry;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.Impl.Cache;
+    using Apache.Ignite.Core.Portable;
+    using Apache.Ignite.Core.Tests.Query;
+    using Apache.Ignite.Core.Transactions;
+    using NUnit.Framework;
+
+    /// <summary>
+    ///
+    /// </summary>
+    class CacheTestKey
+    {
+        /// <summary>
+        /// Default constructor.
+        /// </summary>
+        public CacheTestKey()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="id">ID.</param>
+        public CacheTestKey(int id)
+        {
+            Id = id;
+        }
+
+        /// <summary>
+        /// ID.
+        /// </summary>
+        public int Id
+        {
+            get;
+            set;
+        }
+
+        /** <inheritdoc /> */
+        public override bool Equals(object obj)
+        {
+            CacheTestKey other = obj as CacheTestKey;
+
+            return other != null && Id == other.Id;
+        }
+
+        /** <inheritdoc /> */
+        public override int GetHashCode()
+        {
+            return Id;
+        }
+
+        /** <inheritdoc /> */
+        public override string ToString()
+        {
+            return new StringBuilder()
+                .Append(typeof(CacheTestKey).Name)
+                .Append(" [id=").Append(Id)
+                .Append(']').ToString();
+        }
+    }
+
+    /// <summary>
+    ///
+    /// </summary>
+    class TestReferenceObject
+    {
+        public TestReferenceObject Obj;
+
+        /// <summary>
+        /// Default constructor.
+        /// </summary>
+        public TestReferenceObject()
+        {
+            // No-op.
+        }
+
+        public TestReferenceObject(TestReferenceObject obj)
+        {
+            Obj = obj;
+        }
+    }
+
+    [Serializable]
+    public class TestSerializableObject
+    {
+        public string Name { get; set; }
+        public int Id { get; set; }
+
+        public override bool Equals(object obj)
+        {
+            if (ReferenceEquals(null, obj)) return false;
+            if (ReferenceEquals(this, obj)) return true;
+
+            var other = (TestSerializableObject) obj;
+            return obj.GetType() == GetType() && (string.Equals(Name, other.Name) && Id == other.Id);
+        }
+
+        public override int GetHashCode()
+        {
+            unchecked
+            {
+                return ((Name != null ? Name.GetHashCode() : 0) * 397) ^ Id;
+            }
+        }
+    }
+
+    /// <summary>
+    /// Cache entry processor that adds argument value to the entry value.
+    /// </summary>
+    [Serializable]
+    public class AddArgCacheEntryProcessor : ICacheEntryProcessor<int, int, int, int>
+    {
+        // Expected exception text
+        public const string ExceptionText = "Exception from AddArgCacheEntryProcessor.";
+
+        // Error flag
+        public bool ThrowErr { get; set; }
+
+        // Error flag
+        public bool ThrowErrPortable { get; set; }
+
+        // Error flag
+        public bool ThrowErrNonSerializable { get; set; }
+
+        // Key value to throw error on
+        public int ThrowOnKey { get; set; }
+
+        // Remove flag
+        public bool Remove { get; set; }
+
+        // Exists flag
+        public bool Exists { get; set; }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AddArgCacheEntryProcessor"/> class.
+        /// </summary>
+        public AddArgCacheEntryProcessor()
+        {
+            Exists = true;
+            ThrowOnKey = -1;
+        }
+
+        /** <inheritdoc /> */
+        int ICacheEntryProcessor<int, int, int, int>.Process(IMutableCacheEntry<int, int> entry, int arg)
+        {
+            if (ThrowOnKey < 0 || ThrowOnKey == entry.Key)
+            {
+                if (ThrowErr)
+                    throw new Exception(ExceptionText);
+
+                if (ThrowErrPortable)
+                    throw new PortableTestException {Info = ExceptionText};
+
+                if (ThrowErrNonSerializable)
+                    throw new NonSerializableException();
+            }
+
+            Assert.AreEqual(Exists, entry.Exists);
+
+            if (Remove)
+                entry.Remove();
+            else
+                entry.Value = entry.Value + arg;
+            
+            return entry.Value;
+        }
+
+        /** <inheritdoc /> */
+        public int Process(IMutableCacheEntry<int, int> entry, int arg)
+        {
+            throw new Exception("Invalid method");
+        }
+    }
+
+    /// <summary>
+    /// Portable add processor.
+    /// </summary>
+    public class PortableAddArgCacheEntryProcessor : AddArgCacheEntryProcessor, IPortableMarshalAware
+    {
+        /** <inheritdoc /> */
+        public void WritePortable(IPortableWriter writer)
+        {
+            var w = writer.RawWriter();
+
+            w.WriteBoolean(ThrowErr);
+            w.WriteBoolean(ThrowErrPortable);
+            w.WriteBoolean(ThrowErrNonSerializable);
+            w.WriteInt(ThrowOnKey);
+            w.WriteBoolean(Remove);
+            w.WriteBoolean(Exists);
+        }
+
+        /** <inheritdoc /> */
+        public void ReadPortable(IPortableReader reader)
+        {
+            var r = reader.RawReader();
+
+            ThrowErr = r.ReadBoolean();
+            ThrowErrPortable = r.ReadBoolean();
+            ThrowErrNonSerializable = r.ReadBoolean();
+            ThrowOnKey = r.ReadInt();
+            Remove = r.ReadBoolean();
+            Exists = r.ReadBoolean();
+        }
+    }
+
+    /// <summary>
+    /// Non-serializable processor.
+    /// </summary>
+    public class NonSerializableCacheEntryProcessor : AddArgCacheEntryProcessor
+    {
+        // No-op.
+    }
+
+    /// <summary>
+    /// Portable exception.
+    /// </summary>
+    public class PortableTestException : Exception, IPortableMarshalAware
+    {
+        /// <summary>
+        /// Gets or sets exception info.
+        /// </summary>
+        public string Info { get; set; }
+
+        /** <inheritdoc /> */
+        public override string Message
+        {
+            get { return Info; }
+        }
+
+        /** <inheritdoc /> */
+        public void WritePortable(IPortableWriter writer)
+        {
+            writer.RawWriter().WriteString(Info);
+        }
+
+        /** <inheritdoc /> */
+        public void ReadPortable(IPortableReader reader)
+        {
+            Info = reader.RawReader().ReadString();
+        }
+    }
+
+    /// <summary>
+    /// Non-serializable exception.
+    /// </summary>
+    public class NonSerializableException : Exception
+    {
+        // No-op
+    }
+
+    /// <summary>
+    ///
+    /// </summary>
+    [SuppressMessage("ReSharper", "UnusedVariable")]
+    public abstract class CacheAbstractTest {
+        /// <summary>
+        ///
+        /// </summary>
+        [TestFixtureSetUp]
+        public virtual void StartGrids() {
+            TestUtils.KillProcesses();
+
+            IgniteConfigurationEx cfg = new IgniteConfigurationEx();
+
+            PortableConfiguration portCfg = new PortableConfiguration();
+
+            ICollection<PortableTypeConfiguration> portTypeCfgs = new List<PortableTypeConfiguration>();
+
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(PortablePerson)));
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(CacheTestKey)));
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(TestReferenceObject)));
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(PortableAddArgCacheEntryProcessor)));
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(PortableTestException)));
+
+            portCfg.TypeConfigurations = portTypeCfgs;
+
+            cfg.PortableConfiguration = portCfg;
+            cfg.JvmClasspath = TestUtils.CreateTestClasspath();
+            cfg.JvmOptions = TestUtils.TestJavaOptions();
+            cfg.SpringConfigUrl = "config\\native-client-test-cache.xml";
+
+            for (int i = 0; i < GridCount(); i++) {
+                cfg.GridName = "grid-" + i;
+
+                Ignition.Start(cfg);
+            }
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        [TestFixtureTearDown]
+        public virtual void StopGrids() {
+            for (int i = 0; i < GridCount(); i++)
+                Ignition.Stop("grid-" + i, true);
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        [SetUp]
+        public virtual void BeforeTest()
+        {
+            Console.WriteLine("Test started: " + TestContext.CurrentContext.Test.Name);
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        [TearDown]
+        public virtual void AfterTest() {
+            for (int i = 0; i < GridCount(); i++) 
+                Cache(i).RemoveAll();
+
+            for (int i = 0; i < GridCount(); i++)
+            {
+                var cache = Cache(i);
+
+                if (!cache.IsEmpty)
+                {
+                    var entries = Enumerable.Range(0, 2000)
+                        .Select(x => new KeyValuePair<int, int>(x, cache.LocalPeek(x)))
+                        .Where(x => x.Value != 0)
+                        .Select(pair => pair.ToString() + GetKeyAffinity(cache, pair.Key))
+                        .Aggregate((acc, val) => string.Format("{0}, {1}", acc, val));
+
+                    Assert.Fail("Cache '{0}' is not empty in grid [{1}]: ({2})", CacheName(), i, entries);
+                }
+            }
+
+            Console.WriteLine("Test finished: " + TestContext.CurrentContext.Test.Name);
+        }
+
+        public IIgnite GetIgnite(int idx)
+        {
+            return Ignition.GetIgnite("grid-" + idx);
+        }
+
+        public ICache<int, int> Cache(int idx) {
+            return Cache<int, int>(idx);
+        }
+
+        public ICache<TK, TV> Cache<TK, TV>(int idx) {
+            return GetIgnite(idx).Cache<TK, TV>(CacheName());
+        }
+
+        public ICache<int, int> Cache()
+        {
+            return Cache<int, int>(0);
+        }
+
+        public ICache<TK, TV> Cache<TK, TV>()
+        {
+            return Cache<TK, TV>(0);
+        }
+
+        public ICacheAffinity Affinity()
+        {
+            return GetIgnite(0).Affinity(CacheName());
+        }
+
+        public ITransactions Transactions
+        {
+            get { return GetIgnite(0).Transactions; }
+        }
+
+        [Test]
+        public void TestCircularReference()
+        {
+            var cache = Cache().WithKeepPortable<int, object>();
+
+            TestReferenceObject obj1 = new TestReferenceObject();
+
+            obj1.Obj = new TestReferenceObject(obj1);
+
+            cache.Put(1, obj1);
+
+            var po = (IPortableObject) cache.Get(1);
+
+            Assert.IsNotNull(po);
+
+            TestReferenceObject objRef = po.Deserialize<TestReferenceObject>();
+
+            Assert.IsNotNull(objRef);
+        }
+
+        [Test]
+        public void TestName()
+        {
+            for (int i = 0; i < GridCount(); i++ )
+                Assert.AreEqual(CacheName(), Cache(i).Name);
+        }
+
+        [Test]
+        public void TestIsEmpty()
+        {
+            for (int i = 0; i < GridCount(); i++)
+            {
+                var cache = Cache(i);
+
+                Assert.IsTrue(cache.IsEmpty);
+            }
+
+            for (int i = 0; i < GridCount(); i++)
+            {
+                var cache = Cache(i);
+
+                cache.Put(PrimaryKeyForCache(cache), 1);
+            }
+
+            for (int i = 0; i < GridCount(); i++)
+            {
+                var cache = Cache(i);
+
+                Assert.IsFalse(cache.IsEmpty);
+            }
+        }
+
+        [Test]
+        public void TestContainsKey()
+        {
+            var cache = Cache();
+
+            int key = PrimaryKeyForCache(cache);
+
+            cache.Put(key, 1);
+
+            Assert.IsTrue(cache.ContainsKey(key));
+            Assert.IsFalse(cache.ContainsKey(-1));
+        }
+        
+        [Test]
+        public void TestContainsKeys()
+        {
+            var cache = Cache();
+
+            var keys = PrimaryKeysForCache(cache, 5);
+
+            Assert.IsFalse(cache.ContainsKeys(keys));
+
+            cache.PutAll(keys.ToDictionary(k => k, k => k));
+
+            Assert.IsTrue(cache.ContainsKeys(keys));
+
+            Assert.IsFalse(cache.ContainsKeys(keys.Concat(new[] {int.MaxValue})));
+        }
+        
+        [Test]
+        public void TestPeek()
+        {
+            var cache = Cache();
+
+            int key1 = PrimaryKeyForCache(cache);
+
+            cache.Put(key1, 1);
+
+            Assert.AreEqual(1, cache.LocalPeek(key1));
+            Assert.AreEqual(0, cache.LocalPeek(-1));
+
+            Assert.AreEqual(1, cache.LocalPeek(key1, CachePeekMode.All));
+            Assert.AreEqual(0, cache.LocalPeek(-1, CachePeekMode.All));
+        }
+
+        [Test]
+        public void TestGet()
+        {
+            var cache = Cache();
+
+            cache.Put(1, 1);
+            cache.Put(2, 2);
+
+            Assert.AreEqual(1, cache.Get(1));
+            Assert.AreEqual(2, cache.Get(2));
+            Assert.AreEqual(0, cache.Get(3));
+        }
+
+        [Test]
+        public void TestGetAsync()
+        {
+            var cache = Cache().WithAsync().WrapAsync();
+
+            cache.Put(1, 1);
+            cache.Put(2, 2);
+            
+            Assert.AreEqual(1, cache.Get(1));
+            Assert.AreEqual(2, cache.Get(2));
+            Assert.AreEqual(0, cache.Get(3));
+        }
+
+        [Test]
+        public void TestGetAll()
+        {
+            var cache = Cache();
+
+            cache.Put(1, 1);
+            cache.Put(2, 2);
+            cache.Put(3, 3);
+            cache.Put(4, 4);
+            cache.Put(5, 5);
+
+            IDictionary<int, int> map = cache.GetAll(new List<int> { 0, 1, 2, 5 });
+
+            Assert.AreEqual(3, map.Count);
+
+            Assert.AreEqual(1, map[1]);
+            Assert.AreEqual(2, map[2]);
+        }
+
+        [Test]
+        public void TestGetAllAsync()
+        {
+            var cache = Cache().WithAsync().WrapAsync();
+
+            cache.Put(1, 1);
+            cache.Put(2, 2);
+            cache.Put(3, 3);
+
+            var map = cache.GetAll(new List<int> { 0, 1, 2 });
+
+            Assert.AreEqual(2, map.Count);
+
+            Assert.AreEqual(1, map[1]);
+            Assert.AreEqual(2, map[2]);
+        }
+
+        [Test]
+        public void TestGetAndPut()
+        {
+            var cache = Cache();
+
+            Assert.AreEqual(0, cache.Get(1));
+
+            int old = cache.GetAndPut(1, 1);
+
+            Assert.AreEqual(0, old);
+
+            Assert.AreEqual(1, cache.Get(1));
+
+            old = cache.GetAndPut(1, 2);
+
+            Assert.AreEqual(1, old);
+
+            Assert.AreEqual(2, cache.Get(1));
+        }
+
+        [Test]
+        public void TestGetAndReplace()
+        {
+            var cache = Cache();
+
+            cache.Put(1, 10);
+
+            Assert.AreEqual(10, cache.GetAndReplace(1, 100));
+
+            Assert.AreEqual(0, cache.GetAndReplace(2, 2));
+
+            Assert.AreEqual(0, cache.Get(2));
+
+            Assert.AreEqual(100, cache.Get(1));
+
+            Assert.IsTrue(cache.Remove(1));
+        }
+
+        [Test]
+        public void TestGetAndRemove()
+        {
+            var cache = Cache();
+
+            cache.Put(1, 1);
+
+            Assert.AreEqual(1, cache.Get(1));
+
+            Assert.AreEqual(0, cache.GetAndRemove(0));
+            
+            Assert.AreEqual(1, cache.GetAndRemove(1));
+            
+            Assert.AreEqual(0, cache.GetAndRemove(1));
+            
+            Assert.AreEqual(0, cache.Get(1));
+        }
+
+        [Test]
+        public void TestGetAndPutAsync()
+        {
+            var cache = Cache().WithAsync().WrapAsync();
+
+            Assert.AreEqual(0, cache.Get(1));
+
+            int old = cache.GetAndPut(1, 1);
+
+            Assert.AreEqual(0, old);
+
+            Assert.AreEqual(1, cache.Get(1));
+
+            old = cache.GetAndPut(1, 2);
+
+            Assert.AreEqual(1, old);
+
+            Assert.AreEqual(2, cache.Get(1));
+        }
+
+        [Test]
+        public void TestPut()
+        {
+            var cache = Cache();
+
+            cache.Put(1, 1);
+
+            Assert.AreEqual(1, cache.Get(1));
+        }
+
+        [Test]
+        public void TestPutxAsync()
+        {
+            var cache = Cache().WithAsync().WrapAsync();
+
+            cache.Put(1, 1);
+
+            Assert.AreEqual(1, cache.Get(1));
+        }
+
+        [Test]
+        public void TestPutIfAbsent()
+        {
+            var cache = Cache();
+
+            Assert.AreEqual(0, cache.Get(1));
+
+            Assert.AreEqual(true, cache.PutIfAbsent(1, 1));
+
+            Assert.AreEqual(1, cache.Get(1));
+
+            Assert.AreEqual(false, cache.PutIfAbsent(1, 2));
+
+            Assert.AreEqual(1, cache.Get(1));
+        }
+
+        [Test]
+        public void TestGetAndPutIfAbsent()
+        {
+            var cache = Cache();
+
+            Assert.AreEqual(0, cache.Get(1));
+
+            Assert.AreEqual(0, cache.GetAndPutIfAbsent(1, 1));
+
+            Assert.AreEqual(1, cache.Get(1));
+
+            Assert.AreEqual(1, cache.GetAndPutIfAbsent(1, 2));
+
+            Assert.AreEqual(1, cache.Get(1));
+        }
+
+        [Test]
+        public void TestGetAndPutIfAbsentAsync()
+        {
+            var cache = Cache().WithAsync().WrapAsync();
+
+            Assert.AreEqual(0, cache.Get(1));
+
+            int old = cache.GetAndPutIfAbsent(1, 1);
+
+            Assert.AreEqual(0, old);
+
+            Assert.AreEqual(1, cache.Get(1));
+
+            old = cache.GetAndPutIfAbsent(1, 2);
+
+            Assert.AreEqual(1, old);
+
+            Assert.AreEqual(1, cache.Get(1));
+        }
+
+        [Test]
+        public void TestPutIfAbsentAsync()
+        {
+            var cache = Cache().WithAsync().WrapAsync();
+
+            Assert.AreEqual(0, cache.Get(1));
+
+            Assert.IsTrue(cache.PutIfAbsent(1, 1));
+
+            Assert.AreEqual(1, cache.Get(1));
+
+            Assert.IsFalse(cache.PutIfAbsent(1, 2));
+
+            Assert.AreEqual(1, cache.Get(1));
+        }
+
+        [Test]
+        public void TestReplace()
+        {
+            var cache = Cache();
+
+            Assert.AreEqual(0, cache.Get(1));
+
+            bool success = cache.Replace(1, 1);
+
+            Assert.AreEqual(false, success);
+
+            Assert.AreEqual(0, cache.Get(1));
+
+            cache.Put(1, 1);
+
+            Assert.AreEqual(1, cache.Get(1));
+
+            success = cache.Replace(1, 2);
+
+            Assert.AreEqual(true, success);
+
+            Assert.AreEqual(2, cache.Get(1));
+
+            Assert.IsFalse(cache.Replace(1, -1, 3));
+
+            Assert.AreEqual(2, cache.Get(1));
+
+            Assert.IsTrue(cache.Replace(1, 2, 3));
+
+            Assert.AreEqual(3, cache.Get(1));
+        }
+
+        [Test]
+        public void TestGetAndReplaceAsync()
+        {
+            var cache = Cache().WithAsync().WrapAsync();
+
+            Assert.AreEqual(0, cache.Get(1));
+
+            int old = cache.GetAndReplace(1, 1);
+
+            Assert.AreEqual(0, old);
+
+            Assert.AreEqual(0, cache.Get(1));
+
+            cache.Put(1, 1);
+
+            Assert.AreEqual(1, cache.Get(1));
+
+            old = cache.GetAndReplace(1, 2);
+
+            Assert.AreEqual(1, old);
+
+            Assert.AreEqual(2, cache.Get(1));
+
+            Assert.IsFalse(cache.Replace(1, -1, 3));
+
+            Assert.AreEqual(2, cache.Get(1));
+
+            Assert.IsTrue(cache.Replace(1, 2, 3));
+
+            Assert.AreEqual(3, cache.Get(1));
+        }
+
+        [Test]
+        public void TestReplacex()
+        {
+            var cache = Cache();
+
+            Assert.AreEqual(0, cache.Get(1));
+
+            Assert.IsFalse(cache.Replace(1, 1));
+
+            Assert.AreEqual(0, cache.Get(1));
+
+            cache.Put(1, 1);
+
+            Assert.AreEqual(1, cache.Get(1));
+
+            Assert.IsTrue(cache.Replace(1, 2));
+
+            Assert.AreEqual(2, cache.Get(1));
+        }
+
+        [Test]
+        public void TestReplaceAsync()
+        {
+            var cache = Cache().WithAsync().WrapAsync();
+
+            Assert.AreEqual(0, cache.Get(1));
+
+            Assert.IsFalse(cache.Replace(1, 1));
+
+            Assert.AreEqual(0, cache.Get(1));
+
+            cache.Put(1, 1);
+
+            Assert.AreEqual(1, cache.Get(1));
+
+            Assert.IsTrue(cache.Replace(1, 2));
+
+            Assert.AreEqual(2, cache.Get(1));
+        }
+
+        [Test]
+        public void TestPutAll()
+        {
+            var cache = Cache();
+
+            cache.PutAll(new Dictionary<int, int> { { 1, 1 }, { 2, 2 }, { 3, 3 } });
+
+            Assert.AreEqual(1, cache.Get(1));
+            Assert.AreEqual(2, cache.Get(2));
+            Assert.AreEqual(3, cache.Get(3));
+        }
+
+        [Test]
+        public void TestPutAllAsync()
+        {
+            var cache = Cache().WithAsync().WrapAsync();
+
+            cache.PutAll(new Dictionary<int, int> { { 1, 1 }, { 2, 2 }, { 3, 3 } });
+
+            Assert.AreEqual(1, cache.Get(1));
+            Assert.AreEqual(2, cache.Get(2));
+            Assert.AreEqual(3, cache.Get(3));
+        }
+
+        /// <summary>
+        /// Expiry policy tests.
+        /// </summary>
+        [Test]
+        public void TestWithExpiryPolicy()
+        {
+            ICache<int, int> cache0 = Cache(0);
+            
+            int key0;
+            int key1;
+
+            if (LocalCache())
+            {
+                key0 = 0;
+                key1 = 1;
+            }
+            else
+            {
+                key0 = PrimaryKeyForCache(cache0);
+                key1 = PrimaryKeyForCache(Cache(1));
+            }
+            
+            // Test unchanged expiration.
+            ICache<int, int> cache = cache0.WithExpiryPolicy(new ExpiryPolicy(null, null, null));
+
+            cache.Put(key0, key0);
+            cache.Put(key1, key1);
+            Thread.Sleep(200);
+            Assert.IsTrue(cache0.ContainsKey(key0));
+            Assert.IsTrue(cache0.ContainsKey(key1));
+
+            cache.Put(key0, key0 + 1);
+            cache.Put(key1, key1 + 1);
+            Thread.Sleep(200);
+            Assert.IsTrue(cache0.ContainsKey(key0));
+            Assert.IsTrue(cache0.ContainsKey(key1));
+
+            cache.Get(key0);
+            cache.Get(key1);
+            Thread.Sleep(200);
+            Assert.IsTrue(cache0.ContainsKey(key0));
+            Assert.IsTrue(cache0.ContainsKey(key1));
+
+            cache0.RemoveAll(new List<int> { key0, key1 });
+
+            // Test eternal expiration.
+            cache = cache0.WithExpiryPolicy(new ExpiryPolicy(TimeSpan.MaxValue, TimeSpan.MaxValue, TimeSpan.MaxValue));
+
+            cache.Put(key0, key0);
+            cache.Put(key1, key1);
+            Thread.Sleep(200);
+            Assert.IsTrue(cache0.ContainsKey(key0));
+            Assert.IsTrue(cache0.ContainsKey(key1));
+
+            cache.Put(key0, key0 + 1);
+            cache.Put(key1, key1 + 1);
+            Thread.Sleep(200);
+            Assert.IsTrue(cache0.ContainsKey(key0));
+            Assert.IsTrue(cache0.ContainsKey(key1));
+
+            cache.Get(key0);
+            cache.Get(key1);
+            Thread.Sleep(200);
+            Assert.IsTrue(cache0.ContainsKey(key0));
+            Assert.IsTrue(cache0.ContainsKey(key1));
+
+            cache0.RemoveAll(new List<int> { key0, key1 });
+
+            // Test zero expiration.
+            cache = cache0.WithExpiryPolicy(new ExpiryPolicy(TimeSpan.Zero, TimeSpan.Zero, TimeSpan.Zero));
+
+            cache.Put(key0, key0);
+            cache.Put(key1, key1);
+            Assert.IsFalse(cache0.ContainsKey(key0));
+            Assert.IsFalse(cache0.ContainsKey(key1));
+
+            cache0.Put(key0, key0);
+            cache0.Put(key1, key1);
+            Assert.IsTrue(cache0.ContainsKey(key0));
+            Assert.IsTrue(cache0.ContainsKey(key1));
+            cache.Put(key0, key0 + 1);
+            cache.Put(key1, key1 + 1);
+            Assert.IsFalse(cache0.ContainsKey(key0));
+            Assert.IsFalse(cache0.ContainsKey(key1));
+
+            cache0.Put(key0, key0);
+            cache0.Put(key1, key1);
+            cache.Get(key0);
+            cache.Get(key1);
+            Assert.IsFalse(cache0.ContainsKey(key0));
+            Assert.IsFalse(cache0.ContainsKey(key1));
+
+            cache0.RemoveAll(new List<int> { key0, key1 });
+
+            // Test negative expiration.
+            cache = cache0.WithExpiryPolicy(new ExpiryPolicy(TimeSpan.FromMilliseconds(-100), 
+                TimeSpan.FromMilliseconds(-100), TimeSpan.FromMilliseconds(-100)));
+
+            cache.Put(key0, key0);
+            cache.Put(key1, key1);
+            Assert.IsFalse(cache0.ContainsKey(key0));
+            Assert.IsFalse(cache0.ContainsKey(key1));
+
+            cache0.Put(key0, key0);
+            cache0.Put(key1, key1);
+            Assert.IsTrue(cache0.ContainsKey(key0));
+            Assert.IsTrue(cache0.ContainsKey(key1));
+            cache.Put(key0, key0 + 1);
+            cache.Put(key1, key1 + 1);
+            Assert.IsFalse(cache0.ContainsKey(key0));
+            Assert.IsFalse(cache0.ContainsKey(key1));
+
+            cache0.Put(key0, key0);
+            cache0.Put(key1, key1);
+            cache.Get(key0);
+            cache.Get(key1);
+            Assert.IsFalse(cache0.ContainsKey(key0));
+            Assert.IsFalse(cache0.ContainsKey(key1));
+
+            cache0.RemoveAll(new List<int> { key0, key1 });
+
+            // Test regular expiration.
+            cache = cache0.WithExpiryPolicy(new ExpiryPolicy(TimeSpan.FromMilliseconds(100),
+                TimeSpan.FromMilliseconds(100), TimeSpan.FromMilliseconds(100)));
+
+            cache.Put(key0, key0);
+            cache.Put(key1, key1);
+            Assert.IsTrue(cache0.ContainsKey(key0));
+            Assert.IsTrue(cache0.ContainsKey(key1));
+            Thread.Sleep(200);
+            Assert.IsFalse(cache0.ContainsKey(key0));
+            Assert.IsFalse(cache0.ContainsKey(key1));
+
+            cache0.Put(key0, key0);
+            cache0.Put(key1, key1);
+            cache.Put(key0, key0 + 1);
+            cache.Put(key1, key1 + 1);
+            Assert.IsTrue(cache0.ContainsKey(key0));
+            Assert.IsTrue(cache0.ContainsKey(key1));
+            Thread.Sleep(200);
+            Assert.IsFalse(cache0.ContainsKey(key0));
+            Assert.IsFalse(cache0.ContainsKey(key1));
+
+            cache0.Put(key0, key0);
+            cache0.Put(key1, key1);
+            cache.Get(key0); 
+            cache.Get(key1);
+            Assert.IsTrue(cache0.ContainsKey(key0));
+            Assert.IsTrue(cache0.ContainsKey(key1));
+            Thread.Sleep(200);
+            Assert.IsFalse(cache0.ContainsKey(key0));
+            Assert.IsFalse(cache0.ContainsKey(key1));
+        }
+
+        [Test]
+        public void TestEvict()
+        {
+            var cache = Cache();
+
+            int key = PrimaryKeyForCache(cache);
+
+            cache.Put(key, 1);
+
+            Assert.AreEqual(1, PeekInt(cache, key));
+
+            cache.LocalEvict(new[] {key});
+
+            Assert.AreEqual(0, cache.LocalSize(CachePeekMode.Onheap));
+
+            Assert.AreEqual(0, PeekInt(cache, key));
+
+            Assert.AreEqual(1, cache.Get(key));
+
+            Assert.AreEqual(1, cache.LocalSize(CachePeekMode.Onheap));
+
+            Assert.AreEqual(1, PeekInt(cache, key));
+        }
+
+        [Test]
+        public void TestEvictAllKeys()
+        {
+            var cache = Cache();
+
+            List<int> keys = PrimaryKeysForCache(cache, 3);
+
+            cache.Put(keys[0], 1);
+            cache.Put(keys[1], 2);
+            cache.Put(keys[2], 3);
+
+            Assert.AreEqual(1, PeekInt(cache, keys[0]));
+            Assert.AreEqual(2, PeekInt(cache, keys[1]));
+            Assert.AreEqual(3, PeekInt(cache, keys[2]));
+
+            cache.LocalEvict(new List<int> { -1, keys[0], keys[1] });
+
+            Assert.AreEqual(1, cache.LocalSize(CachePeekMode.Onheap));
+
+            Assert.AreEqual(0, PeekInt(cache, keys[0]));
+            Assert.AreEqual(0, PeekInt(cache, keys[1]));
+            Assert.AreEqual(3, PeekInt(cache, keys[2]));
+
+            Assert.AreEqual(1, cache.Get(keys[0]));
+            Assert.AreEqual(2, cache.Get(keys[1]));
+
+            Assert.AreEqual(3, cache.LocalSize());
+
+            Assert.AreEqual(1, PeekInt(cache, keys[0]));
+            Assert.AreEqual(2, PeekInt(cache, keys[1]));
+            Assert.AreEqual(3, PeekInt(cache, keys[2]));
+        }
+
+        [Test]
+        public void TestClear()
+        {
+            for (int i = 0; i < GridCount(); i++)
+            {
+                var cache = Cache(i);
+
+                cache.Put(PrimaryKeyForCache(cache, 500), 1);
+
+                Assert.IsFalse(cache.IsEmpty);
+            }
+
+            Cache().Clear();
+
+            for (int i = 0; i < GridCount(); i++)
+                Assert.IsTrue(Cache(i).IsEmpty);
+        }
+
+        [Test]
+        public void TestClearKey()
+        {
+            var cache = Cache();
+            var keys = PrimaryKeysForCache(cache, 10);
+
+            foreach (var key in keys)
+                cache.Put(key, 3);
+
+            var i = cache.Size();
+
+            foreach (var key in keys)
+            {
+                cache.Clear(key);
+
+                Assert.AreEqual(0, cache.Get(key));
+
+                Assert.Less(cache.Size(), i);
+
+                i = cache.Size();
+            }
+        }
+
+        [Test]
+        public void TestClearKeys()
+        {
+            var cache = Cache();
+            var keys = PrimaryKeysForCache(cache, 10);
+
+            foreach (var key in keys)
+                cache.Put(key, 3);
+
+            cache.ClearAll(keys);
+
+            foreach (var key in keys)
+                Assert.AreEqual(0, cache.Get(key));
+        }
+
+        [Test]
+        public void TestLocalClearKey()
+        {
+            var cache = Cache();
+            var keys = PrimaryKeysForCache(cache, 10);
+
+            foreach (var key in keys)
+                cache.Put(key, 3);
+
+            var i = cache.Size();
+
+            foreach (var key in keys)
+            {
+                cache.LocalClear(key);
+
+                Assert.AreEqual(0, cache.LocalPeek(key));
+
+                Assert.Less(cache.Size(), i);
+
+                i = cache.Size();
+            }
+
+            cache.Clear();
+        }
+
+        [Test]
+        public void TestLocalClearKeys()
+        {
+            var cache = Cache();
+            var keys = PrimaryKeysForCache(cache, 10);
+
+            foreach (var key in keys)
+                cache.Put(key, 3);
+
+            cache.LocalClearAll(keys);
+
+            foreach (var key in keys)
+                Assert.AreEqual(0, cache.LocalPeek(key));
+
+            cache.Clear();
+        }
+
+        [Test]
+        public void TestRemove()
+        {
+            var cache = Cache();
+
+            cache.Put(1, 1);
+
+            Assert.AreEqual(1, cache.Get(1));
+
+            Assert.AreEqual(true, cache.Remove(1));
+
+            Assert.AreEqual(0, cache.Size());
+
+            Assert.AreEqual(0, cache.Get(1));
+
+            cache.Put(1, 1);
+
+            Assert.AreEqual(1, cache.Get(1));
+
+            Assert.IsFalse(cache.Remove(1, -1));
+            Assert.IsTrue(cache.Remove(1, 1));
+
+            Assert.AreEqual(0, cache.Size());
+
+            Assert.AreEqual(0, cache.Get(1));
+        }
+
+        [Test]
+        public void TestGetAndRemoveAsync()
+        {
+            var cache = Cache().WithAsync().WrapAsync();
+
+            cache.Put(1, 1);
+
+            Assert.AreEqual(1, cache.Get(1));
+
+            Assert.AreEqual(1, cache.GetAndRemove(1));
+
+            Assert.AreEqual(0, cache.Size());
+
+            Assert.AreEqual(0, cache.Get(1));
+
+            cache.Put(1, 1);
+
+            Assert.AreEqual(1, cache.Get(1));
+
+            Assert.IsFalse(cache.Remove(1, -1));
+            Assert.IsTrue(cache.Remove(1, 1));
+
+            Assert.AreEqual(0, cache.Size());
+
+            Assert.AreEqual(0, cache.Get(1));
+        }
+
+        [Test]
+        public void TestRemovex()
+        {
+            var cache = Cache();
+
+            cache.Put(1, 1);
+
+            Assert.AreEqual(1, cache.Get(1));
+
+            Assert.IsFalse(cache.Remove(-1));
+            Assert.IsTrue(cache.Remove(1));
+
+            Assert.AreEqual(0, cache.Size());
+
+            Assert.AreEqual(0, cache.Get(1));
+        }
+
+        [Test]
+        public void TestRemoveAsync()
+        {
+            var cache = Cache().WithAsync().WrapAsync();
+
+            cache.Put(1, 1);
+
+            Assert.AreEqual(1, cache.Get(1));
+
+            Assert.IsFalse(cache.Remove(-1));
+            Assert.IsTrue(cache.Remove(1));
+
+            Assert.AreEqual(0, cache.Size());
+
+            Assert.AreEqual(0, cache.Get(1));
+        }
+
+        [Test]
+        public void TestRemoveAll()
+        {
+            var cache = Cache();
+
+            List<int> keys = PrimaryKeysForCache(cache, 2);
+
+            cache.Put(keys[0], 1);
+            cache.Put(keys[1], 2);
+
+            Assert.AreEqual(1, cache.Get(keys[0]));
+            Assert.AreEqual(2, cache.Get(keys[1]));
+
+            cache.RemoveAll();
+
+            Assert.AreEqual(0, cache.Size());
+
+            Assert.AreEqual(0, cache.Get(keys[0]));
+            Assert.AreEqual(0, cache.Get(keys[1]));
+        }
+
+        [Test]
+        public void TestRemoveAllAsync()
+        {
+            var cache = Cache().WithAsync().WrapAsync();
+
+            List<int> keys = PrimaryKeysForCache(cache, 2);
+
+            cache.Put(keys[0], 1);
+            cache.Put(keys[1], 2);
+
+            Assert.AreEqual(1, cache.Get(keys[0]));
+            Assert.AreEqual(2, cache.Get(keys[1]));
+
+            cache.RemoveAll();
+
+            Assert.AreEqual(0, cache.Size());
+
+            Assert.AreEqual(0, cache.Get(keys[0]));
+            Assert.AreEqual(0, cache.Get(keys[1]));
+        }
+
+        [Test]
+        public void TestRemoveAllKeys()
+        {
+            var cache = Cache();
+
+            Assert.AreEqual(0, cache.Size());
+
+            cache.Put(1, 1);
+            cache.Put(2, 2);
+            cache.Put(3, 3);
+
+            Assert.AreEqual(1, cache.Get(1));
+            Assert.AreEqual(2, cache.Get(2));
+            Assert.AreEqual(3, cache.Get(3));
+
+            cache.RemoveAll(new List<int> { 0, 1, 2 });
+
+            Assert.AreEqual(1, cache.Size(CachePeekMode.Primary));
+
+            Assert.AreEqual(0, cache.Get(1));
+            Assert.AreEqual(0, cache.Get(2));
+            Assert.AreEqual(3, cache.Get(3));
+        }
+
+        [Test]
+        public void TestRemoveAllKeysAsync()
+        {
+            var cache = Cache().WithAsync().WrapAsync();
+
+            Assert.AreEqual(0, cache.Size());
+
+            cache.Put(1, 1);
+            cache.Put(2, 2);
+            cache.Put(3, 3);
+
+            Assert.AreEqual(1, cache.Get(1));
+            Assert.AreEqual(2, cache.Get(2));
+            Assert.AreEqual(3, cache.Get(3));
+
+            cache.RemoveAll(new List<int> { 0, 1, 2 });
+
+            Assert.AreEqual(1, cache.Size(CachePeekMode.Primary));
+
+            Assert.AreEqual(0, cache.Get(1));
+            Assert.AreEqual(0, cache.Get(2));
+            Assert.AreEqual(3, cache.Get(3));
+        }
+
+        [Test]
+        public void TestSizes()
+        {
+            for (int i = 0; i < GridCount(); i++)
+            {
+                var cache = Cache(i);
+
+                List<int> keys = PrimaryKeysForCache(cache, 2);
+
+                foreach (int key in keys)
+                    cache.Put(key, 1);
+
+                Assert.IsTrue(cache.Size() >= 2);
+                Assert.AreEqual(2, cache.LocalSize(CachePeekMode.Primary));
+            }
+
+            ICache<int, int> cache0 = Cache();
+
+            Assert.AreEqual(GridCount() * 2, cache0.Size(CachePeekMode.Primary));
+
+            if (!LocalCache() && !ReplicatedCache())
+            {
+                int nearKey = NearKeyForCache(cache0);
+
+                cache0.Put(nearKey, 1);
+
+                Assert.AreEqual(NearEnabled() ? 1 : 0, cache0.Size(CachePeekMode.Near));
+            }
+        }
+
+        [Test]
+        public void TestLocalSize()
+        {
+            var cache = Cache();
+            var keys = PrimaryKeysForCache(cache, 3);
+
+            cache.Put(keys[0], 1);
+            cache.Put(keys[1], 2);
+
+            var localSize = cache.LocalSize();
+
+            cache.LocalEvict(keys.Take(2).ToArray());
+
+            Assert.AreEqual(0, cache.LocalSize(CachePeekMode.Onheap));
+            Assert.AreEqual(localSize, cache.LocalSize(CachePeekMode.All));
+
+            cache.Put(keys[2], 3);
+
+            Assert.AreEqual(localSize + 1, cache.LocalSize(CachePeekMode.All));
+
+            cache.RemoveAll(keys.Take(2).ToArray());
+        }
+
+        /// <summary>
+        /// Test enumerators.
+        /// </summary>
+        [Test]
+        [SuppressMessage("ReSharper", "PossibleMultipleEnumeration")]
+        public void TestEnumerators()
+        {
+            var cache = Cache();
+            var keys = PrimaryKeysForCache(cache, 2);
+
+            cache.Put(keys[0], keys[0] + 1);
+            cache.Put(keys[1], keys[1] + 1);
+
+            // Check distributed enumerator.
+            IEnumerable<ICacheEntry<int, int>> e = cache;
+
+            CheckEnumerator(e.GetEnumerator(), keys);
+            CheckEnumerator(e.GetEnumerator(), keys);
+
+            // Check local enumerator.
+            e = cache.GetLocalEntries();
+
+            CheckEnumerator(e.GetEnumerator(), keys);
+            CheckEnumerator(e.GetEnumerator(), keys);
+
+            // Evict and check peek modes.
+            cache.LocalEvict(new List<int> { keys[0] } );
+
+            e = cache.GetLocalEntries(CachePeekMode.Onheap);
+            CheckEnumerator(e.GetEnumerator(), new List<int> { keys[1] });
+            CheckEnumerator(e.GetEnumerator(), new List<int> { keys[1] });
+
+            e = cache.GetLocalEntries(CachePeekMode.All);
+            CheckEnumerator(e.GetEnumerator(), keys);
+            CheckEnumerator(e.GetEnumerator(), keys);
+
+            e = cache.GetLocalEntries(CachePeekMode.Onheap, CachePeekMode.Swap);
+            CheckEnumerator(e.GetEnumerator(), keys);
+            CheckEnumerator(e.GetEnumerator(), keys);
+
+            cache.Remove(keys[0]);
+        }
+
+        /// <summary>
+        /// Check enumerator content.
+        /// </summary>
+        /// <param name="e">Enumerator.</param>
+        /// <param name="keys">Keys.</param>
+        private static void CheckEnumerator(IEnumerator<ICacheEntry<int, int>> e, IList<int> keys)
+        {
+            CheckEnumerator0(e, keys);
+
+            e.Reset();
+
+            CheckEnumerator0(e, keys);
+
+            e.Dispose();
+
+            Assert.Throws<ObjectDisposedException>(() => { e.MoveNext(); });
+            Assert.Throws<ObjectDisposedException>(() => { var entry = e.Current; });
+            Assert.Throws<ObjectDisposedException>(e.Reset);
+
+            e.Dispose();
+        }
+
+        /// <summary>
+        /// Check enumerator content.
+        /// </summary>
+        /// <param name="e">Enumerator.</param>
+        /// <param name="keys">Keys.</param>
+        private static void CheckEnumerator0(IEnumerator<ICacheEntry<int, int>> e, IList<int> keys)
+        {
+            Assert.Throws<InvalidOperationException>(() => { var entry = e.Current; });
+
+            int cnt = 0;
+
+            while (e.MoveNext())
+            {
+                ICacheEntry<int, int> entry = e.Current;
+
+                Assert.IsTrue(keys.Contains(entry.Key), "Unexpected entry: " + entry);
+
+                Assert.AreEqual(entry.Key + 1, entry.Value);
+
+                cnt++;
+            }
+
+            Assert.AreEqual(keys.Count, cnt);
+
+            Assert.IsFalse(e.MoveNext());
+
+            Assert.Throws<InvalidOperationException>(() => { var entry = e.Current; });
+        }
+
+        [Test]
+        public void TestPromote()
+        {
+            var cache = Cache();
+
+            int key = PrimaryKeyForCache(cache);
+
+            cache.Put(key, 1);
+
+            Assert.AreEqual(1, PeekInt(cache, key));
+
+            cache.LocalEvict(new[] {key});
+
+            Assert.AreEqual(0, cache.LocalSize(CachePeekMode.Onheap));
+
+            Assert.AreEqual(0, PeekInt(cache, key));
+
+            cache.LocalPromote(new[] { key });
+
+            Assert.AreEqual(1, cache.LocalSize(CachePeekMode.Onheap));
+
+            Assert.AreEqual(1, PeekInt(cache, key));
+        }
+
+        [Test]
+        public void TestPromoteAll()
+        {
+            var cache = Cache();
+
+            List<int> keys = PrimaryKeysForCache(cache, 3);
+
+            cache.Put(keys[0], 1);
+            cache.Put(keys[1], 2);
+            cache.Put(keys[2], 3);
+
+            Assert.AreEqual(1, PeekInt(cache, keys[0]));
+            Assert.AreEqual(2, PeekInt(cache, keys[1]));
+            Assert.AreEqual(3, PeekInt(cache, keys[2]));
+
+            cache.LocalEvict(new List<int> { -1, keys[0], keys[1] });
+
+            Assert.AreEqual(1, cache.LocalSize(CachePeekMode.Onheap));
+
+            Assert.AreEqual(0, PeekInt(cache, keys[0]));
+            Assert.AreEqual(0, PeekInt(cache, keys[1]));
+            Assert.AreEqual(3, PeekInt(cache, keys[2]));
+
+            cache.LocalPromote(new[] {keys[0], keys[1]});
+
+            Assert.AreEqual(3, cache.LocalSize(CachePeekMode.Onheap));
+
+            Assert.AreEqual(1, PeekInt(cache, keys[0]));
+            Assert.AreEqual(2, PeekInt(cache, keys[1]));
+            Assert.AreEqual(3, PeekInt(cache, keys[2]));
+        }
+
+        [Test]
+        public void TestPutGetPortable()
+        {
+            var cache = Cache<int, PortablePerson>();
+
+            PortablePerson obj1 = new PortablePerson("obj1", 1);
+
+            cache.Put(1, obj1);
+
+            obj1 = cache.Get(1);
+
+            Assert.AreEqual("obj1", obj1.Name);
+            Assert.AreEqual(1, obj1.Age);
+        }
+
+        [Test]
+        public void TestPutGetPortableAsync()
+        {
+            var cache = Cache<int, PortablePerson>().WithAsync().WrapAsync();
+
+            PortablePerson obj1 = new PortablePerson("obj1", 1);
+
+            cache.Put(1, obj1);
+
+            obj1 = cache.Get(1);
+
+            Assert.AreEqual("obj1", obj1.Name);
+            Assert.AreEqual(1, obj1.Age);
+        }
+
+        [Test]
+        public void TestPutGetPortableKey()
+        {
+            var cache = Cache<CacheTestKey, string>();
+
+            int cnt = 100;
+
+            for (int i = 0; i < cnt; i++)
+                cache.Put(new CacheTestKey(i), "val-" + i);
+
+            for (int i = 0; i < cnt; i++)
+                Assert.AreEqual("val-" + i, cache.Get(new CacheTestKey(i)));
+        }
+
+        [Test]
+        public void TestGetAsync2()
+        {
+            var cache = Cache().WithAsync();
+
+            for (int i = 0; i < 100; i++)
+            {
+                cache.Put(i, i);
+
+                cache.GetFuture<object>().Get();
+            }
+
+            var futs = new List<IFuture<int>>();
+
+            for (int i = 0; i < 1000; i++)
+            {
+                cache.Get(i % 100);
+
+                futs.Add(cache.GetFuture<int>());
+            }
+
+            for (int i = 0; i < 1000; i++) {
+                Assert.AreEqual(i % 100, futs[i].Get(), "Unexpected result: " + i);
+
+                Assert.IsTrue(futs[i].IsDone);
+            }
+        }
+
+        [Test]
+        [Category(TestUtils.CategoryIntensive)]
+        public void TestGetAsyncMultithreaded()
+        {
+            var cache = Cache().WithAsync();
+
+            for (int i = 0; i < 100; i++)
+            {
+                cache.Put(i, i);
+
+                cache.GetFuture<object>().Get();
+            }
+
+            TestUtils.RunMultiThreaded(() =>
+            {
+                for (int i = 0; i < 100; i++)
+                {
+                    var futs = new List<IFuture<int>>();
+
+                    for (int j = 0; j < 100; j++)
+                    {
+                        cache.Get(j);
+
+                        futs.Add(cache.GetFuture<int>());
+                    }
+
+                    for (int j = 0; j < 100; j++)
+                        Assert.AreEqual(j, futs[j].Get());
+                }
+            }, 10);
+        }
+
+        [Test]
+        [Category(TestUtils.CategoryIntensive)]
+        public void TestPutxAsyncMultithreaded()
+        {
+            var cache = Cache().WithAsync();
+
+            TestUtils.RunMultiThreaded(() =>
+            {
+                Random rnd = new Random();
+
+                for (int i = 0; i < 50; i++)
+                {
+                    var futs = new List<IFuture<object>>();
+
+                    for (int j = 0; j < 10; j++)
+                    {
+                        cache.Put(rnd.Next(1000), i);
+
+                        futs.Add(cache.GetFuture<object>());
+                    }
+
+                    foreach (var fut in futs)
+                        fut.Get();
+                }
+            }, 5);
+        }
+
+        [Test]
+        [Category(TestUtils.CategoryIntensive)]
+        public void TestPutGetAsyncMultithreaded()
+        {
+            var cache = Cache<CacheTestKey, PortablePerson>().WithAsync();
+
+            const int threads = 10;
+            const int objPerThread = 1000;
+
+            int cntr = 0;
+
+            TestUtils.RunMultiThreaded(() =>
+            {
+                // ReSharper disable once AccessToModifiedClosure
+                int threadIdx = Interlocked.Increment(ref cntr);
+
+                var futs = new List<IFuture<object>>();
+
+                for (int i = 0; i < objPerThread; i++)
+                {
+                    int key = threadIdx * objPerThread + i;
+
+                    cache.Put(new CacheTestKey(key), new PortablePerson("Person-" + key, key));
+
+                    futs.Add(cache.GetFuture<object>());
+                }
+
+                foreach (var fut in futs)
+                {
+                    fut.Get();
+
+                    Assert.IsTrue(fut.IsDone);
+                }
+            }, threads);
+
+            for (int i = 0; i < threads; i++)
+            {
+                int threadIdx = i + 1;
+
+                for (int j = 0; j < objPerThread; j++)
+                {
+                    int key = threadIdx * objPerThread + i;
+
+                    cache.Get(new CacheTestKey(key));
+                    var p = cache.GetFuture<PortablePerson>().Get();
+
+                    Assert.IsNotNull(p);
+                    Assert.AreEqual(key, p.Age);
+                    Assert.AreEqual("Person-" + key, p.Name);
+                }
+            }
+
+            cntr = 0;
+
+            TestUtils.RunMultiThreaded(() =>
+            {
+                int threadIdx = Interlocked.Increment(ref cntr);
+
+                for (int i = 0; i < objPerThread; i++)
+                {
+                    int key = threadIdx * objPerThread + i;
+
+                    cache.Put(new CacheTestKey(key), new PortablePerson("Person-" + key, key));
+
+                    cache.GetFuture<object>().Get();
+                }
+            }, threads);
+
+            cntr = 0;
+
+            TestUtils.RunMultiThreaded(() =>
+            {
+                int threadIdx = Interlocked.Increment(ref cntr);
+
+                var futs = new List<IFuture<PortablePerson>>();
+
+                for (int i = 0; i < objPerThread; i++)
+                {
+                    int key = threadIdx * objPerThread + i;
+
+                    cache.Get(new CacheTestKey(key));
+
+                    futs.Add(cache.GetFuture<PortablePerson>());
+                }
+
+                for (int i = 0; i < objPerThread; i++)
+                {
+                    var fut = futs[i];
+
+                    int key = threadIdx * objPerThread + i;
+
+                    var p = fut.Get();
+
+                    Assert.IsNotNull(p);
+                    Assert.AreEqual(key, p.Age);
+                    Assert.AreEqual("Person-" + key, p.Name);
+                }
+            }, threads);
+        }
+
+        //[Test]
+        //[Category(TestUtils.CATEGORY_INTENSIVE)]
+        public void TestAsyncMultithreadedKeepPortable()
+        {
+            var cache = Cache().WithAsync().WithKeepPortable<CacheTestKey, PortablePerson>();
+            var portCache = Cache().WithAsync().WithKeepPortable<CacheTestKey, IPortableObject>();
+
+            const int threads = 10;
+            const int objPerThread = 1000;
+
+            int cntr = 0;
+
+            TestUtils.RunMultiThreaded(() =>
+            {
+                // ReSharper disable once AccessToModifiedClosure
+                int threadIdx = Interlocked.Increment(ref cntr);
+
+                var futs = new List<IFuture<object>>();
+
+                for (int i = 0; i < objPerThread; i++)
+                {
+                    int key = threadIdx * objPerThread + i;
+
+                    cache.Put(new CacheTestKey(key), new PortablePerson("Person-" + key, key));
+
+                    futs.Add(cache.GetFuture<object>());
+                }
+
+                foreach (var fut in futs)
+                    Assert.IsNull(fut.Get());
+            }, threads);
+
+            for (int i = 0; i < threads; i++)
+            {
+                int threadIdx = i + 1;
+
+                for (int j = 0; j < objPerThread; j++)
+                {
+                    int key = threadIdx * objPerThread + i;
+
+                    IPortableObject p = portCache.Get(new CacheTestKey(key));
+
+                    Assert.IsNotNull(p);
+                    Assert.AreEqual(key, p.Field<int>("age"));
+                    Assert.AreEqual("Person-" + key, p.Field<string>("name"));
+                }
+            }
+
+            cntr = 0;
+
+            TestUtils.RunMultiThreaded(() =>
+            {
+                int threadIdx = Interlocked.Increment(ref cntr);
+
+                var futs = new List<IFuture<IPortableObject>>();
+
+                for (int i = 0; i < objPerThread; i++)
+                {
+                    int key = threadIdx * objPerThread + i;
+
+                    portCache.Get(new CacheTestKey(key));
+
+                    futs.Add(cache.GetFuture<IPortableObject>());
+                }
+
+                for (int i = 0; i < objPerThread; i++)
+                {
+                    var fut = futs[i];
+
+                    int key = threadIdx * objPerThread + i;
+
+                    var p = fut.Get();
+
+                    Assert.IsNotNull(p);
+                    Assert.AreEqual(key, p.Field<int>("age"));
+                    Assert.AreEqual("Person-" + key, p.Field<string>("name"));
+                }
+            }, threads);
+
+            cntr = 0;
+
+            TestUtils.RunMultiThreaded(() =>
+            {
+                int threadIdx = Interlocked.Increment(ref cntr);
+
+                var futs = new List<IFuture<bool>>();
+
+                for (int i = 0; i < objPerThread; i++)
+                {
+                    int key = threadIdx * objPerThread + i;
+
+                    cache.Remove(new CacheTestKey(key));
+
+                    futs.Add(cache.GetFuture<bool>());
+                }
+
+                for (int i = 0; i < objPerThread; i++)
+                {
+                    var fut = futs[i];
+
+                    Assert.AreEqual(true, fut.Get());
+                }
+            }, threads);
+        }
+
+        [Test]
+        [Ignore("IGNITE-835")]
+        public void TestLock()
+        {
+            if (!LockingEnabled())
+                return;
+
+            var cache = Cache();
+
+            const int key = 7;
+
+            // Lock
+            CheckLock(cache, key, () => cache.Lock(key));
+
+            // LockAll
+            CheckLock(cache, key, () => cache.LockAll(new[] { key, 2, 3, 4, 5 }));
+        }
+
+        /// <summary>
+        /// Internal lock test routine.
+        /// </summary>
+        /// <param name="cache">Cache.</param>
+        /// <param name="key">Key.</param>
+        /// <param name="getLock">Function to get the lock.</param>
+        private static void CheckLock(ICache<int, int> cache, int key, Func<ICacheLock> getLock)
+        {
+            var sharedLock = getLock();
+            
+            using (sharedLock)
+            {
+                Assert.Throws<InvalidOperationException>(() => sharedLock.Exit());  // can't exit if not entered
+
+                sharedLock.Enter();
+
+                try
+                {
+                    Assert.IsTrue(cache.IsLocalLocked(key, true));
+                    Assert.IsTrue(cache.IsLocalLocked(key, false));
+
+                    EnsureCannotLock(getLock, sharedLock);
+
+                    sharedLock.Enter();
+
+                    try
+                    {
+                        Assert.IsTrue(cache.IsLocalLocked(key, true));
+                        Assert.IsTrue(cache.IsLocalLocked(key, false));
+
+                        EnsureCannotLock(getLock, sharedLock);
+                    }
+                    finally
+                    {
+                        sharedLock.Exit();
+                    }
+
+                    Assert.IsTrue(cache.IsLocalLocked(key, true));
+                    Assert.IsTrue(cache.IsLocalLocked(key, false));
+
+                    EnsureCannotLock(getLock, sharedLock);
+
+                    Assert.Throws<SynchronizationLockException>(() => sharedLock.Dispose()); // can't dispose while locked
+                }
+                finally
+                {
+                    sharedLock.Exit();
+                }
+
+                Assert.IsFalse(cache.IsLocalLocked(key, true));
+                Assert.IsFalse(cache.IsLocalLocked(key, false));
+
+                var innerTask = new Task(() =>
+                {
+                    Assert.IsTrue(sharedLock.TryEnter());
+                    sharedLock.Exit();
+
+                    using (var otherLock = getLock())
+                    {
+                        Assert.IsTrue(otherLock.TryEnter());
+                        otherLock.Exit();
+                    }
+                });
+
+                innerTask.Start();
+                innerTask.Wait();
+            }
+            
+            Assert.IsFalse(cache.IsLocalLocked(key, true));
+            Assert.IsFalse(cache.IsLocalLocked(key, false));
+            
+            var outerTask = new Task(() =>
+            {
+                using (var otherLock = getLock())
+                {
+                    Assert.IsTrue(otherLock.TryEnter());
+                    otherLock.Exit();
+                }
+            });
+
+            outerTask.Start();
+            outerTask.Wait();
+
+            Assert.Throws<ObjectDisposedException>(() => sharedLock.Enter());  // Can't enter disposed lock
+        }
+
+        /// <summary>
+        /// ENsure taht lock cannot be obtained by other threads.
+        /// </summary>
+        /// <param name="getLock">Get lock function.</param>
+        /// <param name="sharedLock">Shared lock.</param>
+        private static void EnsureCannotLock(Func<ICacheLock> getLock, ICacheLock sharedLock)
+        {
+            var task = new Task(() =>
+            {
+                Assert.IsFalse(sharedLock.TryEnter());
+                Assert.IsFalse(sharedLock.TryEnter(TimeSpan.FromMilliseconds(100)));
+
+                using (var otherLock = getLock())
+                {
+                    Assert.IsFalse(otherLock.TryEnter());
+                    Assert.IsFalse(otherLock.TryEnter(TimeSpan.FromMilliseconds(100)));
+                }
+            });
+
+            task.Start();
+            task.Wait();
+        }
+
+        [Test]
+        public void TestTxCommit()
+        {
+            TestTxCommit(false);
+        }
+
+        [Test]
+        public void TestTxCommitAsync()
+        {
+            TestTxCommit(true);
+        }
+
+        private void TestTxCommit(bool async)
+        {
+            if (!TxEnabled())
+                return;
+
+            var cache = Cache();
+
+            ITransaction tx = Transactions.Tx;
+
+            Assert.IsNull(tx);
+
+            tx = Transactions.TxStart();
+
+            try
+            {
+                cache.Put(1, 1);
+
+                cache.Put(2, 2);
+
+                if (async)
+                {
+                    var asyncTx = tx.WithAsync();
+                    
+                    asyncTx.Commit();
+
+                    var fut = asyncTx.GetFuture();
+
+                    fut.Get();
+
+                    Assert.IsTrue(fut.IsDone);
+                    Assert.AreEqual(fut.Get(), null);
+                }
+                else
+                    tx.Commit();
+            }
+            finally
+            {
+                tx.Dispose();
+            }
+
+            Assert.AreEqual(1, cache.Get(1));
+
+            Assert.AreEqual(2, cache.Get(2));
+
+            tx = Transactions.Tx;
+
+            Assert.IsNull(tx);
+        }
+
+        [Test]
+        public void TestTxRollback()
+        {
+            if (!TxEnabled())
+                return;
+
+            var cache = Cache();
+
+            cache.Put(1, 1);
+
+            cache.Put(2, 2);
+
+            ITransaction tx = Transactions.Tx;
+
+            Assert.IsNull(tx);
+
+            tx = Transactions.TxStart();
+
+            try {
+                cache.Put(1, 10);
+
+                cache.Put(2, 20);
+            }
+            finally {
+                tx.Rollback();
+            }
+
+            Assert.AreEqual(1, cache.Get(1));
+
+            Assert.AreEqual(2, cache.Get(2));
+
+            Assert.IsNull(Transactions.Tx);
+        }
+
+        [Test]
+        public void TestTxClose()
+        {
+            if (!TxEnabled())
+                return;
+
+            var cache = Cache();
+
+            cache.Put(1, 1);
+
+            cache.Put(2, 2);
+
+            ITransaction tx = Transactions.Tx;
+
+            Assert.IsNull(tx);
+
+            tx = Transactions.TxStart();
+
+            try
+            {
+                cache.Put(1, 10);
+
+                cache.Put(2, 20);
+            }
+            finally
+            {
+                tx.Dispose();
+            }
+
+            Assert.AreEqual(1, cache.Get(1));
+
+            Assert.AreEqual(2, cache.Get(2));
+
+            tx = Transactions.Tx;
+
+            Assert.IsNull(tx);
+        }
+        
+        [Test]
+        public void TestTxAllModes()
+        {
+            TestTxAllModes(false);
+
+            TestTxAllModes(true);
+
+            Console.WriteLine("Done");
+        }
+
+        protected void TestTxAllModes(bool withTimeout)
+        {
+            if (!TxEnabled())
+                return;
+
+            var cache = Cache();
+
+            int cntr = 0;
+
+            foreach (TransactionConcurrency concurrency in Enum.GetValues(typeof(TransactionConcurrency))) {
+                foreach (TransactionIsolation isolation in Enum.GetValues(typeof(TransactionIsolation))) {
+                    Console.WriteLine("Test tx [concurrency=" + concurrency + ", isolation=" + isolation + "]");
+
+                    ITransaction tx = Transactions.Tx;
+
+                    Assert.IsNull(tx);
+
+                    tx = withTimeout 
+                        ? Transactions.TxStart(concurrency, isolation, TimeSpan.FromMilliseconds(1100), 10)
+                        : Transactions.TxStart(concurrency, isolation);
+
+                    Assert.AreEqual(concurrency, tx.Concurrency);
+                    Assert.AreEqual(isolation, tx.Isolation);
+
+                    if (withTimeout)
+                        Assert.AreEqual(1100, tx.Timeout.TotalMilliseconds);
+
+                    try {
+                        cache.Put(1, cntr);
+
+                        tx.Commit();
+                    }
+                    finally {
+                        tx.Dispose();
+                    }
+
+                    tx = Transactions.Tx;
+
+                    Assert.IsNull(tx);
+
+                    Assert.AreEqual(cntr, cache.Get(1));
+
+                    cntr++;
+                }
+            }
+        }
+
+        [Test]
+        public void TestTxAttributes()
+        {
+            if (!TxEnabled())
+                return;
+
+            ITransaction tx = Transactions.TxStart(TransactionConcurrency.Optimistic,
+                TransactionIsolation.RepeatableRead, TimeSpan.FromMilliseconds(2500), 100);
+
+            Assert.IsFalse(tx.IsRollbackOnly);
+            Assert.AreEqual(TransactionConcurrency.Optimistic, tx.Concurrency);
+            Assert.AreEqual(TransactionIsolation.RepeatableRead, tx.Isolation);
+            Assert.AreEqual(2500, tx.Timeout.TotalMilliseconds);
+            Assert.AreEqual(TransactionState.Active, tx.State);
+            Assert.IsTrue(tx.StartTime.Ticks > 0);
+            Assert.AreEqual(tx.NodeId, GetIgnite(0).Cluster.LocalNode.Id);
+
+            DateTime startTime1 = tx.StartTime;
+
+            tx.Commit();
+
+            Assert.IsFalse(tx.IsRollbackOnly);
+            Assert.AreEqual(TransactionState.Committed, tx.State);
+            Assert.AreEqual(TransactionConcurrency.Optimistic, tx.Concurrency);
+            Assert.AreEqual(TransactionIsolation.RepeatableRead, tx.Isolation);
+            Assert.AreEqual(2500, tx.Timeout.TotalMilliseconds);
+            Assert.AreEqual(startTime1, tx.StartTime);
+
+            Thread.Sleep(100);
+
+            tx = Transactions.TxStart(TransactionConcurrency.Pessimistic, TransactionIsolation.ReadCommitted,
+                TimeSpan.FromMilliseconds(3500), 200);
+
+            Assert.IsFalse(tx.IsRollbackOnly);
+            Assert.AreEqual(TransactionConcurrency.Pessimistic, tx.Concurrency);
+            Assert.AreEqual(TransactionIsolation.ReadCommitted, tx.Isolation);
+            Assert.AreEqual(3500, tx.Timeout.TotalMilliseconds);
+            Assert.AreEqual(TransactionState.Active, tx.State);
+            Assert.IsTrue(tx.StartTime.Ticks > 0);
+            Assert.IsTrue(tx.StartTime > startTime1);
+
+            DateTime startTime2 = tx.StartTime;
+
+            tx.Rollback();
+
+            Assert.AreEqual(TransactionState.RolledBack, tx.State);
+            Assert.AreEqual(TransactionConcurrency.Pessimistic, tx.Concurrency);
+            Assert.AreEqual(TransactionIsolation.ReadCommitted, tx.Isolation);
+            Assert.AreEqual(3500, tx.Timeout.TotalMilliseconds);
+            Assert.AreEqual(startTime2, tx.StartTime);
+
+            Thread.Sleep(100);
+
+            tx = Transactions.TxStart(TransactionConcurrency.Optimistic, TransactionIsolation.RepeatableRead,
+                TimeSpan.FromMilliseconds(2500), 100);
+
+            Assert.IsFalse(tx.IsRollbackOnly);
+            Assert.AreEqual(TransactionConcurrency.Optimistic, tx.Concurrency);
+            Assert.AreEqual(TransactionIsolation.RepeatableRead, tx.Isolation);
+            Assert.AreEqual(2500, tx.Timeout.TotalMilliseconds);
+            Assert.AreEqual(TransactionState.Active, tx.State);
+            Assert.IsTrue(tx.StartTime > startTime2);
+
+            DateTime startTime3 = tx.StartTime;
+
+            tx.Commit();
+
+            Assert.IsFalse(tx.IsRollbackOnly);
+            Assert.AreEqual(TransactionState.Committed, tx.State);
+            Assert.AreEqual(TransactionConcurrency.Optimistic, tx.Concurrency);
+            Assert.AreEqual(TransactionIsolation.RepeatableRead, tx.Isolation);
+            Assert.AreEqual(2500, tx.Timeout.TotalMilliseconds);
+            Assert.AreEqual(startTime3, tx.StartTime);
+        }
+
+        [Test]
+        public void TestTxRollbackOnly()
+        {
+            if (!TxEnabled())
+                return;
+
+            var cache = Cache();
+
+            cache.Put(1, 1);
+
+            cache.Put(2, 2);
+
+            ITransaction tx = Transactions.TxStart();
+
+            cache.Put(1, 10);
+
+            cache.Put(2, 20);
+
+            Assert.IsFalse(tx.IsRollbackOnly);
+
+            tx.SetRollbackonly();
+
+            Assert.IsTrue(tx.IsRollbackOnly);
+
+            Assert.AreEqual(TransactionState.MarkedRollback, tx.State);
+
+            try
+            {
+                tx.Commit();
+
+                Assert.Fail("Commit must fail.");
+            }
+            catch (IgniteException e)
+            {
+                Console.WriteLine("Expected exception: " + e);
+            }
+
+            tx.Dispose();
+
+            Assert.AreEqual(TransactionState.RolledBack, tx.State);
+
+            Assert.IsTrue(tx.IsRollbackOnly);
+
+            Assert.AreEqual(1, cache.Get(1));
+
+            Assert.AreEqual(2, cache.Get(2));
+
+            tx = Transactions.Tx;
+
+            Assert.IsNull(tx);
+        }
+
+        [Test]
+        public void TestTxMetrics()
+        {
+            if (!TxEnabled())
+                return;
+
+            var cache = Cache();
+            
+            var startTime = DateTime.UtcNow.AddSeconds(-1);
+
+            Transactions.ResetMetrics();
+
+            var metrics = Transactions.GetMetrics();
+            
+            Assert.AreEqual(0, metrics.TxCommits);
+            Assert.AreEqual(0, metrics.TxRollbacks);
+
+            using (Transactions.TxStart())
+            {
+                cache.Put(1, 1);
+            }
+            
+            using (var tx = Transactions.TxStart())
+            {
+                cache.Put(1, 1);
+                tx.Commit();
+            }
+
+            metrics = Transactions.GetMetrics();
+
+            Assert.AreEqual(1, metrics.TxCommits);
+            Assert.AreEqual(1, metrics.TxRollbacks);
+
+            Assert.LessOrEqual(startTime, metrics.CommitTime);
+            Assert.LessOrEqual(startTime, metrics.RollbackTime);
+
+            Assert.GreaterOrEqual(DateTime.UtcNow, metrics.CommitTime);
+            Assert.GreaterOrEqual(DateTime.UtcNow, metrics.RollbackTime);
+        }
+
+        [Test]
+        public void TestTxStateAndExceptions()
+        {
+            if (!TxEnabled())
+                return;
+
+            var tx = Transactions.TxStart();
+            
+            Assert.AreEqual(TransactionState.Active, tx.State);
+
+            tx.Rollback();
+
+            Assert.AreEqual(TransactionState.RolledBack, tx.State);
+
+            try
+            {
+                tx.Commit();
+                Assert.Fail();
+            }
+            catch (InvalidOperationException)
+            {
+                // Expected
+            }
+
+            tx = Transactions.TxStart().WithAsync();
+
+            Assert.AreEqual(TransactionState.Active, tx.State);
+
+            tx.Commit();
+
+            tx.GetFuture().Get();
+
+            Assert.AreEqual(TransactionState.Committed, tx.State);
+
+            tx.Rollback();  // Illegal, but should not fail here; will fail in future
+
+            try
+            {
+                tx.GetFuture<object>().Get();
+                Assert.Fail();
+            }
+            catch (InvalidOperationException)
+            {
+                // Expected
+            }
+        }
+        
+        /// <summary>
+        /// Test thraed-locals leak.
+        /// </summary>
+        [Test]
+        [Category(TestUtils.CategoryIntensive)]
+        public void TestThreadLocalLeak()
+        {
+            var cache = Cache<string, string>();
+
+            Exception err = null;
+
+            const int threadCnt = 10;
+
+            Thread[] threads = new Thread[threadCnt];
+
+            ThreadStart[] threadStarts = new ThreadStart[threadCnt];
+
+            for (int j = 0; j < threadCnt; j++)
+            {
+                string key = "key" + j;
+
+                threadStarts[j] = () =>
+                {
+                    try
+                    {
+                        cache.Put(key, key);
+
+                        Assert.AreEqual(key, cache.Get(key));
+                    }
+                    catch (Exception e)
+                    {
+                        Interlocked.CompareExchange(ref err, e, null);
+
+                        Assert.Fail("Unexpected error: " + e);
+                    }
+                };
+            }
+
+            for (int i = 0; i < 100 && err == null; i++)
+            {
+                for (int j = 0 ; j < threadCnt; j++) {
+                    Thread t = new Thread(threadStarts[j]);
+
+                    threads[j] = t;
+                }
+
+                foreach (Thread t in threads)
+                    t.Start();
+
+                foreach (Thread t in threads)
+                    t.Join();
+
+                if (i % 500 == 0)
+                {
+                    Console.WriteLine("Iteration: " + i);
+
+                    GC.Collect();
+                }
+            }
+
+            Assert.IsNull(err);
+        }
+
+        //// TODO [Test]
+        //public void TestCacheEntryGetSetRemove()
+        //{
+        //    TestCacheEntryGetSetRemove(false);
+        //}
+
+        //// TODO [Test]
+        //public void TestCacheEntryGetSetRemoveAsync()
+        //{
+        //    TestCacheEntryGetSetRemove(true);
+        //}
+
+        //private void TestCacheEntryGetSetRemove(bool async)
+        //{
+        //    IGridCache cache = Cache();
+
+        //    int key = PrimaryKeyForCache(cache);
+
+        //    IGridCacheEntry<int, int> e = cache.Entry(key);
+
+        //    Assert.IsNotNull(e);
+
+        //    if (async)
+        //    {
+        //        IGridFuture<int> fut = e.GetAsync();
+
+        //        Assert.IsNull(fut.Result);
+
+        //        Assert.IsTrue(fut.IsDone);
+
+        //        fut = e.SetAsync(10);
+
+        //        Assert.IsNull(fut.Result);
+
+        //        Assert.IsTrue(fut.IsDone);
+
+        //        fut = e.GetAsync();
+
+        //        Assert.AreEqual(10, fut.Result);
+
+        //        Assert.IsTrue(fut.IsDone);
+        //    }
+        //    else
+        //    {
+        //        Assert.IsNull(e.Get());
+
+        //        Assert.IsNull(e.Set(10));
+
+        //        Assert.AreEqual(10, e.Get());
+        //    }
+
+        //    Assert.AreEqual(10, cache.Get(key));
+
+        //    if (async)
+        //    {
+        //        IGridFuture<int> fut = e.RemoveAsync();
+
+        //        Assert.AreEqual(10, fut.Result);
+
+        //        Assert.IsTrue(fut.IsDone);
+        //    }
+        //    else
+        //        Assert.AreEqual(10, e.Remove());
+
+        //    Assert.IsNull(e.Get());
+
+        //    Assert.IsNull(cache.Get(key));
+        //}
+
+        /**
+         * Test tries to provoke garbage collection for .Net future before it was completed to verify
+         * futures pinning works.
+         */
+        [Test]
+        [Category(TestUtils.CategoryIntensive)]
+        public void TestFuturesGc()
+        {
+            var cache = Cache().WithAsync();
+
+            cache.Put(1, 1);
+
+            for (int i = 0; i < 10; i++)
+            {
+                TestUtils.RunMultiThreaded(() =>
+                {
+                    for (int j = 0; j < 1000; j++)
+                        cache.Get(1);
+                }, 5);
+
+                GC.Collect();
+
+                cache.Get(1);
+                Assert.AreEqual(1, cache.GetFuture<int>().Get());
+            }
+
+            Thread.Sleep(2000);
+        }
+
+        [Test]
+        public void TestPartitions()
+        {
+            ICacheAffinity aff = Affinity();
+
+            for (int i = 0; i < 5; i++ )
+                Assert.AreEqual(CachePartitions(), aff.Partitions);
+        }
+
+        [Test]
+        public void TestKeyPartition()
+        {
+            ICacheAffinity aff = Affinity();
+
+            {
+                ISet<int> parts = new HashSet<int>();
+
+                for (int i = 0; i < 1000; i++)
+                    parts.Add(aff.Partition(i));
+
+                if (LocalCache())
+                    Assert.AreEqual(1, parts.Count);
+                else
+                    Assert.IsTrue(parts.Count > 10);
+            }
+
+            {
+                ISet<int> parts = new HashSet<int>();
+
+                for (int i = 0; i < 1000; i++)
+                    parts.Add(aff.Partition("key" + i));
+
+                if (LocalCache())
+                    Assert.AreEqual(1, parts.Count);
+                else
+                    Assert.IsTrue(parts.Count > 10);
+            }
+        }
+
+        [Test]
+        public void TestIsPrimaryOrBackup()
+        {
+            ICacheAffinity aff = Affinity();
+
+            ICollection<IClusterNode> nodes = GetIgnite(0).Cluster.Nodes();
+
+            Assert.IsTrue(nodes.Count > 0);
+
+            IClusterNode node = nodes.First();
+
+            {
+                bool found = false;
+
+                for (int i = 0; i < 1000; i++)
+                {
+                    if (aff.IsPrimary(node, i))
+                    {
+                        Assert.IsTrue(aff.IsPrimaryOrBackup(node, i));
+
+                        found = true;
+
+                        if (nodes.Count > 1)
+                            Assert.IsFalse(aff.IsPrimary(nodes.Last(), i));
+
+                        break;
+                    }
+                }
+
+                Assert.IsTrue(found, "Failed to find primary key for node " + node);
+            }
+
+            if (nodes.Count > 1)
+            {
+                bool found = false;
+
+                for (int i = 0; i < 1000; i++)
+                {
+                    if (aff.IsBackup(node, i))
+                    {
+                        Assert.IsTrue(aff.IsPrimaryOrBackup(node, i));
+
+                        found = true;
+
+                        break;
+                    }
+                }
+
+                Assert.IsTrue(found, "Failed to find backup key for node " + node);
+            }
+        }
+
+        [Test]
+        public void TestNodePartitions()
+        {
+            ICacheAffinity aff = Affinity();
+
+            ICollection<IClusterNode> nodes = GetIgnite(0).Cluster.Nodes();
+
+            Assert.IsTrue(nodes.Count > 0);
+
+            if (nodes.Count == 1)
+            {
+                IClusterNode node = nodes.First();
+
+                int[] parts = aff.BackupPartitions(node);
+
+                Assert.AreEqual(0, parts.Length);
+
+                parts = aff.AllPartitions(node);
+
+                Assert.AreEqual(CachePartitions(), parts.Length);
+            }
+            else
+            {
+                IList<int> allPrimaryParts = new List<int>();
+                IList<int> allBackupParts = new List<int>();
+                IList<int> allParts = new List<int>();
+
+                foreach(IClusterNode node in nodes) {
+                    int[] parts = aff.PrimaryPartitions(node);
+
+                    foreach (int part in parts)
+                        allPrimaryParts.Add(part);
+
+                    parts = aff.BackupPartitions(node);
+
+                    foreach (int part in parts)
+                        allBackupParts.Add(part);
+
+                    parts = aff.AllPartitions(node);
+
+                    foreach (int part in parts)
+                        allParts.Add(part);
+                }
+
+                Assert.AreEqual(CachePartitions(), allPrimaryParts.Count);
+                Assert.AreEqual(CachePartitions() * Backups(), allBackupParts.Count);
+                Assert.AreEqual(CachePartitions() * (Backups() + 1), allParts.Count);
+            }
+        }
+
+        [Test]
+        public void TestAffinityKey()
+        {
+            ICacheAffinity aff = Affinity();
+
+            Assert.AreEqual(10, aff.AffinityKey<int, int>(10));
+
+            Assert.AreEqual("string", aff.AffinityKey<string, string>("string"));
+        }
+
+        [Test]
+        public void TestMapToNode()
+        {
+            ICacheAffinity aff = Affinity();
+
+            const int key = 1;
+
+            IClusterNode node = aff.MapKeyToNode(key);
+
+            Assert.IsNotNull(node);
+
+            Assert.IsTrue(GetIgnite(0).Cluster.Nodes().Contains(node));
+
+            Assert.IsTrue(aff.IsPrimary(node, key));
+
+            Assert.IsTrue(aff.IsPrimaryOrBackup(node, key));
+
+            Assert.IsFalse(aff.IsBackup(node, key));
+
+            int part = aff.Partition(key);
+
+            IClusterNode partNode = aff.MapPartitionToNode(part);
+
+            Assert.AreEqual(node, partNode);
+        }
+
+        [Test]
+        public void TestMapToPrimaryAndBackups()
+        {
+            ICacheAffinity aff = Affinity();
+
+            const int key = 1;
+
+            IList<IClusterNode> nodes = aff.MapKeyToPrimaryAndBackups(key);
+
+            Assert.IsTrue(nodes.Count > 0);
+
+            for (int i = 0; i < nodes.Count; i++)
+            {
+                if (i == 0)
+                    Assert.IsTrue(aff.IsPrimary(nodes[i], key));
+                else
+                    Assert.IsTrue(aff.IsBackup(nodes[i], key));
+            }
+
+            int part = aff.Partition(key);
+
+            IList<IClusterNode> partNodes = aff.MapPartitionToPrimaryAndBackups(part);
+
+            Assert.AreEqual(nodes, partNodes);
+        }
+
+        [Test]
+        public void TestMapKeysToNodes()
+        {
+            ICacheAffinity aff = Affinity();
+
+            IList<int> keys = new List<int> {1, 2, 3};
+
+            IDictionary<IClusterNode, IList<int>> map = aff.MapKeysToNodes(keys);
+
+            Assert.IsTrue(map.Count > 0);
+
+            foreach (int key in keys)
+            {
+                IClusterNode primary = aff.MapKeyToNode(key);
+
+                Assert.IsTrue(map.ContainsKey(primary));
+
+                IList<int> nodeKeys = map[primary];
+
+                Assert.IsNotNull(nodeKeys);
+
+                Assert.IsTrue(nodeKeys.Contains(key));
+            }
+        }
+
+        [Test]
+        public void TestMapPartitionsToNodes()
+        {
+            ICacheAffinity aff = Affinity();
+
+            if (LocalCache())
+            {
+                IList<int> parts = new List<int> { 0 };
+
+                IDictionary<int, IClusterNode> map = aff.MapPartitionsToNodes(parts);
+
+                Assert.AreEqual(parts.Count, map.Count);
+
+                Assert.AreEqual(GetIgnite(0).Cluster.LocalNode, map[0]);
+            }
+            else
+            {
+                IList<int> parts = new List<int> { 1, 2, 3 };
+
+                IDictionary<int, IClusterNode> map = aff.MapPartitionsToNodes(parts);
+
+                Assert.AreEqual(parts.Count, map.Count);
+
+                foreach (int part in parts)
+                {
+                    Assert.IsTrue(map.ContainsKey(part));
+
+                    IClusterNode primary = aff.MapPartitionToNode(part);
+
+                    Assert.AreEqual(primary, map[part], "Wrong node for partition: " + part);
+                }
+            }
+        }
+
+        [Test]
+        public void TestKeepPortableFlag()
+        {
+            TestKeepPortableFlag(false);
+        }
+
+        [Test]
+        public void TestKeepPortableFlagAsync()
+        {
+            TestKeepPortableFlag(true);
+        }
+
+        [Test]
+        public void TestNearKeys()
+        {
+            if (!NearEnabled())
+                return;
+
+            const int count = 20;
+
+            var cache = Cache();
+            var aff = cache.Ignite.Affinity(cache.Name);
+            var node = cache.Ignite.Cluster.LocalNode;
+
+            for (int i = 0; i < count; i++)
+                cache.Put(i, -i - 1);
+
+            var nearKeys = Enumerable.Range(0, count).Where(x => !aff.IsPrimaryOrBackup(node, x)).ToArray();
+
+            var nearKeysString = nearKeys.Select(x => x.ToString()).Aggregate((x, y) => x + ", " + y);
+
+            Console.WriteLine("Near keys: " + nearKeysString);
+
+            foreach (var nearKey in nearKeys.Take(3))
+                Assert.AreNotEqual(0, cache.Get(nearKey));
+        }
+        
+        [Test]
+        public void TestSerializable()
+        {
+            var cache = Cache<int, TestSerializableObject>();
+
+            var obj = new TestSerializableObject {Name = "Vasya", Id = 128};
+
+            cache.Put(1, obj);
+
+            var resultObj = cache.Get(1);
+
+            Assert.AreEqual(obj, resultObj);
+        }
+
+        [Test]
+        public void TestInvoke()
+        {
+            TestInvoke(false);
+        }
+
+        [Test]
+        public void TestInvokeAsync()
+        {
+            TestInvoke(true);
+        }
+
+        private void TestInvoke(bool async)
+        {
+            TestInvoke<AddArgCacheEntryProcessor>(async);
+            TestInvoke<PortableAddArgCacheEntryProcessor>(async);
+
+            try
+            {
+                TestInvoke<NonSerializableCacheEntryProcessor>(async);
+                Assert.Fail();
+            }
+            catch (SerializationException)
+            {
+                // Expected
+            }
+        }
+
+        private void TestInvoke<T>(bool async) where T: AddArgCacheEntryProcessor, new()
+        {
+            var cache = async ? Cache().WithAsync().WrapAsync() : Cache();
+
+            cache.Clear();
+
+            const int key = 1;
+            const int value = 3;
+            const int arg = 5;
+
+            cache.Put(key, value);
+
+            // Existing entry
+            Assert.AreEqual(value + arg, cache.Invoke(key, new T(), arg));
+            Assert.AreEqual(value + arg, cache.Get(key));
+
+            // Non-existing entry
+            Assert.AreEqual(arg, cache.Invoke(10, new T {Exists = false}, arg));
+            Assert.AreEqual(arg, cache.Get(10));
+
+            // Remove entry
+            Assert.AreEqual(0, cache.Invoke(key, new T {Remove = true}, arg));
+            Assert.AreEqual(0, cache.Get(key));
+
+            // Test exceptions
+            AssertThrowsCacheEntryProcessorException(() => cache.Invoke(key, new T {ThrowErr = true}, arg));
+            AssertThrowsCacheEntryProcessorException(
+                () => cache.Invoke(key, new T {ThrowErrPortable = true}, arg));
+            AssertThrowsCacheEntryProcessorException(
+                () => cache.Invoke(key, new T { ThrowErrNonSerializable = true }, arg), "SerializationException");
+        }
+
+        private static void AssertThrowsCacheEntryProcessorException(Action action, string containsText = null)
+        {
+            try
+            {
+                action();
+
+                Assert.Fail();
+            }
+            catch (Exception ex)
+            {
+                Assert.IsInstanceOf<CacheEntryProcessorException>(ex);
+
+                if (string.IsNullOrEmpty(containsText))
+                    Assert.AreEqual(ex.InnerException.Message, AddArgCacheEntryProcessor.ExceptionText);
+                else
+                    Assert.IsTrue(ex.ToString().Contains(containsText));
+            }
+        }
+
+        [Test]
+        public void TestInvokeAll()
+        {
+            TestInvokeAll(false);
+        }
+
+        [Test]
+        public void TestInvokeAllAsync()
+        {
+            TestInvokeAll(true);
+        }
+
+        private void TestInvokeAll(bool async)
+        {
+            for (var i = 1; i < 10; i++)
+            {
+                TestInvokeAll<AddArgCacheEntryProcessor>(async, i);
+                TestInvokeAll<PortableAddArgCacheEntryProcessor>(async, i);
+
+                try
+                {
+                    TestInvokeAll<NonSerializableCacheEntryProcessor>(async, i);
+                    Assert.Fail();
+                }
+                catch (SerializationException)
+                {
+                    // Expected
+                }
+            }
+        }
+
+        public void TestInvokeAll<T>(bool async, int entryCount) where T : AddArgCacheEntryProcessor, new()
+        {
+            var cache = async ? Cache().WithAsync().WrapAsync() : Cache();
+
+            var entries = Enumerable.Range(1, entryCount).ToDictionary(x => x, x => x + 1);
+
+            cache.PutAll(entries);
+
+            const int arg = 5;
+
+            // Existing entries
+            var res = cache.InvokeAll(entries.Keys, new T(), arg);
+
+            var results = res.OrderBy(x => x.Key).Select(x => x.Value.Result);
+            var expectedResults = entries.OrderBy(x => x.Key).Select(x => x.Value + arg);
+            
+            Assert.IsTrue(results.SequenceEqual(expectedResults));
+
+            var resultEntries = cache.GetAll(entries.Keys);
+
+            Assert.IsTrue(resultEntries.All(x => x.Value == x.Key + 1 + arg));
+
+            // Remove entries
+            res = cache.InvokeAll(entries.Keys, new T {Remove = true}, arg);
+
+            Assert.IsTrue(res.All(x => x.Value.Result == 0));
+            Assert.AreEqual(0, cache.GetAll(entries.Keys).Count);
+
+            // Non-existing entries
+            res = cache.InvokeAll(entries.Keys, new T {Exists = false}, arg);
+
+            Assert.IsTrue(res.All(x => x.Value.Result == arg));
+            Assert.IsTrue(cache.GetAll(entries.Keys).All(x => x.Value == arg)); 
+
+            // Test exceptions
+            var errKey = entries.Keys.Reverse().Take(5).Last();
+
+            TestInvokeAllException(cache, entries, new T { ThrowErr = true, ThrowOnKey = errKey }, arg, errKey);
+            TestInvokeAllException(cache, entries, new T { ThrowErrPortable = true, ThrowOnKey = errKey }, 
+                arg, errKey);
+            TestInvokeAllException(cache, entries, new T { ThrowErrNonSerializable = true, ThrowOnKey = errKey }, 
+                arg, errKey, "SerializationException");
+
+        }
+
+        private static void TestInvokeAllException<T>(ICache<int, int> cache, Dictionary<int, int> entries, 
+            T processor, int arg, int errKey, string exceptionText = null) where T : AddArgCacheEntryProcessor
+        {
+            var res = cache.InvokeAll(entries.Keys, processor, arg);
+
+            foreach (var procRes in res)
+            {
+                if (procRes.Key == errKey)
+                    // ReSharper disable once AccessToForEachVariableInClosure
+                    AssertThrowsCacheEntryProcessorException(() => { var x = procRes.Value.Result; }, exceptionText);
+                else
+                    Assert.Greater(procRes.Value.Result, 0);
+            }
+        }
+
+        /// <summary>
+        /// Test skip-store semantics.
+        /// </summary>
+        [Test]
+        public void TestSkipStore()
+        {
+            CacheProxyImpl<int, int> cache = (CacheProxyImpl<int, int>)Cache();
+
+            Assert.IsFalse(cache.SkipStore);
+
+            // Ensure correct flag set.
+ 

<TRUNCATED>

[09/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/TaskResultTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/TaskResultTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/TaskResultTest.cs
new file mode 100644
index 0000000..b67e854
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/TaskResultTest.cs
@@ -0,0 +1,437 @@
+/*
+ * 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.Compute
+{
+    using System;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Portable;
+    using Apache.Ignite.Core.Resource;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests task result.
+    /// </summary>
+    public class TaskResultTest : AbstractTaskTest
+    {
+        /** Grid name. */
+        private static volatile string _gridName;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        public TaskResultTest() : base(false) { }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="forked">Fork flag.</param>
+        protected TaskResultTest(bool forked) : base(forked) { }
+
+        /// <summary>
+        /// Test for task result.
+        /// </summary>
+        [Test]
+        public void TestTaskResultInt()
+        {
+            TestTask<int> task = new TestTask<int>();
+
+            int res = Grid1.Compute().Execute(task, new Tuple<bool, int>(true, 10));
+
+            Assert.AreEqual(10, res);
+
+            res = Grid1.Compute().Execute(task, new Tuple<bool, int>(false, 11));
+
+            Assert.AreEqual(11, res);
+        }
+
+        /// <summary>
+        /// Test for task result.
+        /// </summary>
+        [Test]
+        public void TestTaskResultLong()
+        {
+            TestTask<long> task = new TestTask<long>();
+
+            long res = Grid1.Compute().Execute(task, new Tuple<bool, long>(true, 10000000000));
+
+            Assert.AreEqual(10000000000, res);
+
+            res = Grid1.Compute().Execute(task, new Tuple<bool, long>(false, 10000000001));
+
+            Assert.AreEqual(10000000001, res);
+        }
+
+        /// <summary>
+        /// Test for task result.
+        /// </summary>
+        [Test]
+        public void TestTaskResultFloat()
+        {
+            TestTask<float> task = new TestTask<float>();
+
+            float res = Grid1.Compute().Execute(task, new Tuple<bool, float>(true, 1.1f));
+
+            Assert.AreEqual(1.1f, res);
+
+            res = Grid1.Compute().Execute(task, new Tuple<bool, float>(false, -1.1f));
+
+            Assert.AreEqual(-1.1f, res);
+        }
+
+        /// <summary>
+        /// Test for task result.
+        /// </summary>
+        [Test]
+        public void TestTaskResultPortable()
+        {
+            TestTask<PortableResult> task = new TestTask<PortableResult>();
+
+            PortableResult val = new PortableResult(100);
+
+            PortableResult res = Grid1.Compute().Execute(task, new Tuple<bool, PortableResult>(true, val));
+
+            Assert.AreEqual(val.Val, res.Val);
+
+            val.Val = 101;
+
+            res = Grid1.Compute().Execute(task, new Tuple<bool, PortableResult>(false, val));
+
+            Assert.AreEqual(val.Val, res.Val);
+        }
+
+        /// <summary>
+        /// Test for task result.
+        /// </summary>
+        [Test]
+        public void TestTaskResultSerializable()
+        {
+            TestTask<SerializableResult> task = new TestTask<SerializableResult>();
+
+            SerializableResult val = new SerializableResult(100);
+
+            SerializableResult res = Grid1.Compute().Execute(task, new Tuple<bool, SerializableResult>(true, val));
+
+            Assert.AreEqual(val.Val, res.Val);
+
+            val.Val = 101;
+
+            res = Grid1.Compute().Execute(task, new Tuple<bool, SerializableResult>(false, val));
+
+            Assert.AreEqual(val.Val, res.Val);
+        }
+
+        /// <summary>
+        /// Test for task result.
+        /// </summary>
+        [Test]
+        public void TestTaskResultLarge()
+        {
+            TestTask<byte[]> task = new TestTask<byte[]>();
+
+            byte[] res = Grid1.Compute().Execute(task,
+                new Tuple<bool, byte[]>(true, new byte[100 * 1024]));
+
+            Assert.AreEqual(100 * 1024, res.Length);
+
+            res = Grid1.Compute().Execute(task, new Tuple<bool, byte[]>(false, new byte[101 * 1024]));
+
+            Assert.AreEqual(101 * 1024, res.Length);
+        }
+
+        /** <inheritDoc /> */
+        override protected void PortableTypeConfigurations(ICollection<PortableTypeConfiguration> portTypeCfgs)
+        {
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(PortableResult)));
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(TestPortableJob)));
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(PortableOutFunc)));
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof(PortableFunc)));
+        }
+
+        [Test]
+        public void TestOutFuncResultPrimitive1()
+        {
+            ICollection<int> res = Grid1.Compute().Broadcast(new PortableOutFunc());
+
+            Assert.AreEqual(3, res.Count);
+
+            foreach (int r in res)
+                Assert.AreEqual(10, r);
+        }
+
+        [Test]
+        public void TestOutFuncResultPrimitive2()
+        {
+            ICollection<int> res = Grid1.Compute().Broadcast(new SerializableOutFunc());
+
+            Assert.AreEqual(3, res.Count);
+
+            foreach (int r in res)
+                Assert.AreEqual(10, r);
+        }
+
+        [Test]
+        public void TestFuncResultPrimitive1()
+        {
+            ICollection<int> res = Grid1.Compute().Broadcast(new PortableFunc(), 10);
+
+            Assert.AreEqual(3, res.Count);
+
+            foreach (int r in res)
+                Assert.AreEqual(11, r);
+        }
+
+        [Test]
+        public void TestFuncResultPrimitive2()
+        {
+            ICollection<int> res = Grid1.Compute().Broadcast(new SerializableFunc(), 10);
+
+            Assert.AreEqual(3, res.Count);
+
+            foreach (int r in res)
+                Assert.AreEqual(11, r);
+        }
+
+        interface IUserInterface<in T, out TR>
+        {
+            TR Invoke(T arg);
+        }
+
+        /// <summary>
+        /// Test function.
+        /// </summary>
+        public class PortableFunc : IComputeFunc<int, int>, IUserInterface<int, int>
+        {
+            int IComputeFunc<int, int>.Invoke(int arg)
+            {
+                return arg + 1;
+            }
+
+            int IUserInterface<int, int>.Invoke(int arg)
+            {
+                // Same signature as IComputeFunc<int, int>, but from different interface
+                throw new Exception("Invalid method");
+            }
+
+            public int Invoke(int arg)
+            {
+                // Same signature as IComputeFunc<int, int>, 
+                // but due to explicit interface implementation this is a wrong method
+                throw new Exception("Invalid method");
+            }
+        }
+
+        /// <summary>
+        /// Test function.
+        /// </summary>
+        [Serializable]
+        public class SerializableFunc : IComputeFunc<int, int>
+        {
+            public int Invoke(int arg)
+            {
+                return arg + 1;
+            }
+        }
+
+        /// <summary>
+        /// Test function.
+        /// </summary>
+        public class PortableOutFunc : IComputeFunc<int>
+        {
+            public int Invoke()
+            {
+                return 10;
+            }
+        }
+
+        /// <summary>
+        /// Test function.
+        /// </summary>
+        [Serializable]
+        public class SerializableOutFunc : IComputeFunc<int>
+        {
+            public int Invoke()
+            {
+                return 10;
+            }
+        }
+
+        /// <summary>
+        /// Test task.
+        /// </summary>
+        public class TestTask<T> : ComputeTaskAdapter<Tuple<bool, T>, T, T>
+        {
+            /** <inheritDoc /> */
+            override public IDictionary<IComputeJob<T>, IClusterNode> Map(IList<IClusterNode> subgrid, Tuple<bool, T> arg)
+            {
+                _gridName = null;
+
+                Assert.AreEqual(3, subgrid.Count);
+
+                bool local = arg.Item1;
+                T res = arg.Item2;
+
+                var jobs = new Dictionary<IComputeJob<T>, IClusterNode>();
+
+                IComputeJob<T> job;
+
+                if (res is PortableResult)
+                {
+                    TestPortableJob job0 = new TestPortableJob();
+
+                    job0.SetArguments(res);
+
+                    job = (IComputeJob<T>) job0;
+                }
+                else
+                {
+                    TestJob<T> job0 = new TestJob<T>();
+
+                    job0.SetArguments(res);
+
+                    job = job0;
+                }
+
+                foreach (IClusterNode node in subgrid)
+                {
+                    bool add = local ? node.IsLocal : !node.IsLocal;
+
+                    if (add)
+                    {
+                        jobs.Add(job, node);
+
+                        break;
+                    }
+                }
+
+                Assert.AreEqual(1, jobs.Count);
+
+                return jobs;
+            }
+
+            /** <inheritDoc /> */
+            override public T Reduce(IList<IComputeJobResult<T>> results)
+            {
+                Assert.AreEqual(1, results.Count);
+
+                var res = results[0];
+
+                Assert.IsNull(res.Exception());
+
+                Assert.IsFalse(res.Cancelled);
+
+                Assert.IsNotNull(_gridName);
+
+                Assert.AreEqual(GridId(_gridName), res.NodeId);
+
+                var job = res.Job();
+
+                Assert.IsNotNull(job);
+
+                return res.Data();
+            }
+        }
+
+        private static Guid GridId(string gridName)
+        {
+            if (gridName.Equals(Grid1Name))
+                return Ignition.GetIgnite(Grid1Name).Cluster.LocalNode.Id;
+            if (gridName.Equals(Grid2Name))
+                return Ignition.GetIgnite(Grid2Name).Cluster.LocalNode.Id;
+            if (gridName.Equals(Grid3Name))
+                return Ignition.GetIgnite(Grid3Name).Cluster.LocalNode.Id;
+
+            Assert.Fail("Failed to find grid " + gridName);
+
+            return new Guid();
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        class PortableResult
+        {
+            /** */
+            public int Val;
+
+            public PortableResult(int val)
+            {
+                Val = val;
+            }
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        [Serializable]
+        class SerializableResult
+        {
+            /** */
+            public int Val;
+
+            public SerializableResult(int val)
+            {
+                Val = val;
+            }
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        [Serializable]
+        class TestJob<T> : ComputeJobAdapter<T>
+        {
+            [InstanceResource]
+            private IIgnite _grid = null;
+
+            /** <inheritDoc /> */
+            override public T Execute()
+            {
+                Assert.IsNotNull(_grid);
+
+                _gridName = _grid.Name;
+
+                T res = Argument<T>(0);
+
+                return res;
+            }
+        }
+
+        /// <summary>
+        ///
+        /// </summary>
+        class TestPortableJob : ComputeJobAdapter<PortableResult>
+        {
+            [InstanceResource]
+            private IIgnite _grid = null;
+
+            /** <inheritDoc /> */
+            override public PortableResult Execute()
+            {
+                Assert.IsNotNull(_grid);
+
+                _gridName = _grid.Name;
+
+                PortableResult res = Argument<PortableResult>(0);
+
+                return res;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Cache/Store/cache-store-session.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Cache/Store/cache-store-session.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Cache/Store/cache-store-session.xml
new file mode 100644
index 0000000..00837a9
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Cache/Store/cache-store-session.xml
@@ -0,0 +1,80 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+    <bean id="storeFactory" class="org.apache.ignite.platform.dotnet.PlatformDotNetCacheStoreFactory">
+        <property name="assemblyName" value="Apache.Ignite.Core.Tests"/>
+        <property name="className" value="Apache.Ignite.Core.Tests.Cache.Store.CacheStoreSessionTest+Store"/>
+    </bean>
+  
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+
+        <property name="includeEventTypes">
+            <util:constant static-field="org.apache.ignite.events.EventType.EVTS_CACHE"/>
+        </property>
+      
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache1"/>
+                    <property name="cacheMode" value="LOCAL"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="writeThrough" value="true"/>
+                    <property name="readThrough" value="true"/>
+
+                    <property name="cacheStoreFactory" ref="storeFactory" />
+                </bean>
+
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache2"/>
+                    <property name="cacheMode" value="LOCAL"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="writeThrough" value="true"/>
+                    <property name="readThrough" value="true"/>
+
+                    <property name="cacheStoreFactory" ref="storeFactory" />
+                </bean>
+            </list>
+        </property>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47501</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
new file mode 100644
index 0000000..183676b
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
@@ -0,0 +1,90 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+      <property name="localHost" value="127.0.0.1"/>
+      <property name="connectorConfiguration"><null/></property>
+
+      <property name="gridName" value="grid1"/>
+
+      <property name="metricsUpdateFrequency" value="1000"/>
+        <property name="metricsLogFrequency" value="0"/>
+
+        <property name="userAttributes">
+            <map>
+                <entry key="my_attr" value="value1"/>
+            </map>
+        </property>
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="startSize" value="10"/>
+                </bean>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache1"/>
+                    <property name="startSize" value="10"/>
+                </bean>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache2"/>
+                    <property name="startSize" value="10"/>
+                </bean>
+            </list>
+        </property>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47502</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+
+        <!-- Portable marshaller configuration -->
+        <property name="marshaller">
+            <bean class="org.apache.ignite.marshaller.portable.PortableMarshaller">
+                <property name="typeConfigurations">
+                    <list>
+                        <bean class="org.apache.ignite.portable.PortableTypeConfiguration">
+                            <property name="className" value="org.apache.ignite.platform.PlatformComputePortable"/>
+                        </bean>
+                        <bean class="org.apache.ignite.portable.PortableTypeConfiguration">
+                            <property name="className" value="org.apache.ignite.platform.PlatformComputeJavaPortable"/>
+                        </bean>
+                        <bean class="org.apache.ignite.portable.PortableTypeConfiguration">
+                            <property name="className" value="org.apache.ignite.platform.PlatformComputeEnum"/>
+                        </bean>
+                    </list>
+                </property>
+            </bean>
+        </property>
+
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid2.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid2.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid2.xml
new file mode 100644
index 0000000..434f468
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid2.xml
@@ -0,0 +1,63 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+      <property name="localHost" value="127.0.0.1"/>
+      <property name="connectorConfiguration"><null/></property>
+
+      <property name="gridName" value="grid2"/>
+
+      <property name="metricsUpdateFrequency" value="1000"/>
+        <property name="metricsLogFrequency" value="0"/>
+
+        <property name="userAttributes">
+            <map>
+                <entry key="my_attr" value="value2"/>
+            </map>
+        </property>
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache1"/>
+                    <property name="startSize" value="10"/>
+                </bean>
+            </list>
+        </property>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47502</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid3.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid3.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid3.xml
new file mode 100644
index 0000000..31ccdf0
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid3.xml
@@ -0,0 +1,52 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+      <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+
+      <property name="clientMode" value="true"/>
+
+      <property name="gridName" value="grid3"/>
+
+      <property name="metricsUpdateFrequency" value="1000"/>
+      <property name="metricsLogFrequency" value="0"/>
+
+      <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="forceServerMode" value="true"/>
+
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47502</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-standalone.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-standalone.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-standalone.xml
new file mode 100644
index 0000000..bd34958
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-standalone.xml
@@ -0,0 +1,87 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+
+        <property name="metricsLogFrequency" value="0"/>
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration" />
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache1"/>
+                </bean>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache2"/>
+                </bean>
+            </list>
+        </property>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <value>127.0.0.1:47500..47502</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+
+        <property name="platformConfiguration">
+            <bean class="org.apache.ignite.platform.dotnet.PlatformDotNetConfiguration">
+                <property name="portableConfiguration">
+                    <bean class="org.apache.ignite.platform.dotnet.PlatformDotNetPortableConfiguration">
+                        <property name="types">
+                            <list>
+                                <value>Apache.Ignite.Core.Tests.ExecutableTest+RemoteConfiguration</value>
+                                <value>Apache.Ignite.Core.Tests.ExecutableTest+RemoteConfigurationClosure</value>
+                                <value>Apache.Ignite.Core.Tests.Compute.TaskAdapterTest+PortableJob</value>
+                                <value>Apache.Ignite.Core.Tests.Compute.PortableClosureTaskTest+PortableOutFunc</value>
+                                <value>Apache.Ignite.Core.Tests.Compute.PortableClosureTaskTest+PortableFunc</value>
+                                <value>Apache.Ignite.Core.Tests.Compute.PortableClosureTaskTest+PortableResult</value>
+                                <value>Apache.Ignite.Core.Tests.Compute.PortableClosureTaskTest+PortableException</value>
+                            </list>
+                        </property>
+                        <property name="typesConfiguration">
+                            <list>
+                                <bean class="org.apache.ignite.platform.dotnet.PlatformDotNetPortableTypeConfiguration">
+                                    <property name="typeName" value="org.apache.ignite.platform.PlatformComputePortable"/>
+                                </bean>
+                                <bean class="org.apache.ignite.platform.dotnet.PlatformDotNetPortableTypeConfiguration">
+                                    <property name="typeName" value="org.apache.ignite.platform.PlatformComputeJavaPortable"/>
+                                </bean>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Dynamic/dynamic-client.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Dynamic/dynamic-client.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Dynamic/dynamic-client.xml
new file mode 100644
index 0000000..8f8893f
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Dynamic/dynamic-client.xml
@@ -0,0 +1,51 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+
+        <property name="clientMode" value="true"/>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="forceServerMode" value="true"/>
+
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47502</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Dynamic/dynamic-data-no-cfg.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Dynamic/dynamic-data-no-cfg.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Dynamic/dynamic-data-no-cfg.xml
new file mode 100644
index 0000000..83c6642
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Dynamic/dynamic-data-no-cfg.xml
@@ -0,0 +1,47 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47502</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Dynamic/dynamic-data.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Dynamic/dynamic-data.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Dynamic/dynamic-data.xml
new file mode 100644
index 0000000..dbbae90
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Dynamic/dynamic-data.xml
@@ -0,0 +1,65 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="p"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="startSize" value="10"/>
+                </bean>
+
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="pa"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="startSize" value="10"/>
+                </bean>
+            </list>
+        </property>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47502</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Ignite.exe.config.test
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Ignite.exe.config.test b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Ignite.exe.config.test
new file mode 100644
index 0000000..305fbd5
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Ignite.exe.config.test
@@ -0,0 +1,41 @@
+<?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>
+    <startup>
+        <supportedRuntime version="v4.0" sku=".NETFramework,Version=v4.0"/>
+    </startup>
+
+    <runtime>
+        <gcServer enabled="true" />
+    </runtime>
+
+    <appSettings>
+        <add key="GridGain.SpringConfigUrl" value="config\compute\compute-standalone.xml" />
+        <add key="GridGain.Assembly.1" value="test-1.dll" />
+        <add key="GridGain.Assembly.2" value="test-2.dll" />
+        <add key="GridGain.JvmOption.1" value="-DOPT1" />
+        <add key="GridGain.JvmOption.2" value="-DOPT2" />
+        <add key="GridGain.JvmOption.3" value="-Xms601m" />
+        <add key="GridGain.JvmOption.4" value="-Xmx702m" />
+        <add key="GridGain.JvmInitialMemoryMB" value="601" />
+        <add key="GridGain.JvmMaxMemoryMB" value="702" />
+    </appSettings>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Lifecycle/lifecycle-beans.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Lifecycle/lifecycle-beans.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Lifecycle/lifecycle-beans.xml
new file mode 100644
index 0000000..da36032
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Lifecycle/lifecycle-beans.xml
@@ -0,0 +1,66 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+      <property name="localHost" value="127.0.0.1"/>
+      <property name="connectorConfiguration"><null/></property>
+
+        <property name="gridName" value="grid"/>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <value>127.0.0.1:47500..47502</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+
+        <property name="lifecycleBeans">
+            <list>
+                <bean class="org.apache.ignite.platform.lifecycle.PlatformJavaLifecycleBean" />
+                <bean class="org.apache.ignite.platform.dotnet.PlatformDotNetLifecycleBean">
+                    <property name="className" value="Apache.Ignite.Core.Tests.Bean" />
+                </bean>
+                <bean class="org.apache.ignite.platform.lifecycle.PlatformJavaLifecycleBean" />
+                <bean class="org.apache.ignite.platform.dotnet.PlatformDotNetLifecycleBean">
+                    <property name="className" value="Apache.Ignite.Core.Tests.Bean" />
+                    <property name="properties">
+                        <map>
+                            <entry key="Property1">
+                                <value type="java.lang.Integer">1</value>
+                            </entry>
+                            <entry key="Property2" value="1"/>
+                        </map>
+                    </property>
+                </bean>
+                <bean class="org.apache.ignite.platform.lifecycle.PlatformJavaLifecycleBean" />
+            </list>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Lifecycle/lifecycle-no-beans.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Lifecycle/lifecycle-no-beans.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Lifecycle/lifecycle-no-beans.xml
new file mode 100644
index 0000000..4063e6e
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Lifecycle/lifecycle-no-beans.xml
@@ -0,0 +1,44 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+      <property name="localHost" value="127.0.0.1"/>
+      <property name="connectorConfiguration"><null/></property>
+
+      <property name="gridName" value="grid"/>
+
+      <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <value>127.0.0.1:47500..47502</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/cache-portables.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/cache-portables.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/cache-portables.xml
new file mode 100644
index 0000000..84f9e5a
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/cache-portables.xml
@@ -0,0 +1,78 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+
+        <property name="platformConfiguration">
+            <bean class="org.apache.ignite.platform.dotnet.PlatformDotNetConfiguration">
+                <property name="portableConfiguration">
+                    <bean class="org.apache.ignite.platform.dotnet.PlatformDotNetPortableConfiguration">
+                        <property name="types">
+                            <list>
+                                <value>Apache.Ignite.Core.Tests.TestGenericPortable[System.Int64]</value>
+                                <value>Apache.Ignite.Core.Tests.TestGenericPortable[System.Type]</value>
+                                <value>Apache.Ignite.Core.Tests.TestGenericPortable[Apache.Ignite.Core.Tests.TestGenericPortable[System.Int64]]</value>
+                                <value>Apache.Ignite.Core.Tests.TestGenericPortable[System.Collections.Generic.List[System.Tuple[System.Int64,System.String]]]</value>
+                                <value>Apache.Ignite.Core.Tests.TestGenericPortable[System.Int64,System.String]</value>
+                                <value>Apache.Ignite.Core.Tests.TestGenericPortable[System.Int64,Apache.Ignite.Core.Tests.TestGenericPortable[System.String]]</value>
+                                <value>Apache.Ignite.Core.Tests.TestGenericPortable[System.Int64,System.String,System.Type]</value>
+                                <value>Apache.Ignite.Core.Tests.TestGenericPortable[System.Int64,System.String,Apache.Ignite.Core.Tests.TestGenericPortable[System.Int64,System.String,System.Type]]</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+
+        <property name="includeEventTypes">
+            <util:constant static-field="org.apache.ignite.events.EventType.EVTS_CACHE"/>
+        </property>
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration" />
+            </list>
+        </property>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47501</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/cache-query-continuous.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/cache-query-continuous.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/cache-query-continuous.xml
new file mode 100644
index 0000000..7f9ce40
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/cache-query-continuous.xml
@@ -0,0 +1,171 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="transactional_no_backup"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
+                    <property name="backups" value="0"/>
+                    <property name="startSize" value="10"/>
+                    <property name="typeMetadata">
+                        <list>
+                            <bean class="org.apache.ignite.cache.CacheTypeMetadata">
+                                <property name="valueType" value="PortableEntry"/>
+                                <property name="ascendingFields">
+                                    <map>
+                                        <entry key="val" value="java.lang.Integer"/>
+                                    </map>
+                                </property>
+                                <property name="queryFields">
+                                    <map>
+                                        <entry key="val" value="java.lang.Integer"/>
+                                    </map>
+                                </property>
+                                <property name="textFields">
+                                    <list>
+                                        <value>val</value>
+                                    </list>
+                                </property>
+                            </bean>
+                        </list>
+                    </property>
+                </bean>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="transactional_backup"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
+                    <property name="backups" value="1"/>
+                    <property name="startSize" value="10"/>
+                    <property name="typeMetadata">
+                        <list>
+                            <bean class="org.apache.ignite.cache.CacheTypeMetadata">
+                                <property name="valueType" value="PortableEntry"/>
+                                <property name="ascendingFields">
+                                    <map>
+                                        <entry key="val" value="java.lang.Integer"/>
+                                    </map>
+                                </property>
+                                <property name="queryFields">
+                                    <map>
+                                        <entry key="val" value="java.lang.Integer"/>
+                                    </map>
+                                </property>
+                                <property name="textFields">
+                                    <list>
+                                        <value>val</value>
+                                    </list>
+                                </property>
+                            </bean>
+                        </list>
+                    </property>
+                </bean>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="atomic_no_backup"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
+                    <property name="backups" value="0"/>
+                    <property name="startSize" value="10"/>
+                    <property name="typeMetadata">
+                        <list>
+                            <bean class="org.apache.ignite.cache.CacheTypeMetadata">
+                                <property name="valueType" value="PortableEntry"/>
+                                <property name="ascendingFields">
+                                    <map>
+                                        <entry key="val" value="java.lang.Integer"/>
+                                    </map>
+                                </property>
+                                <property name="queryFields">
+                                    <map>
+                                        <entry key="val" value="java.lang.Integer"/>
+                                    </map>
+                                </property>
+                                <property name="textFields">
+                                    <list>
+                                        <value>val</value>
+                                    </list>
+                                </property>
+                            </bean>
+                        </list>
+                    </property>
+                </bean>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="atomic_backup"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
+                    <property name="backups" value="1"/>
+                    <property name="startSize" value="10"/>
+                    <property name="typeMetadata">
+                        <list>
+                            <bean class="org.apache.ignite.cache.CacheTypeMetadata">
+                                <property name="valueType" value="PortableEntry"/>
+                                <property name="ascendingFields">
+                                    <map>
+                                        <entry key="val" value="java.lang.Integer"/>
+                                    </map>
+                                </property>
+                                <property name="queryFields">
+                                    <map>
+                                        <entry key="val" value="java.lang.Integer"/>
+                                    </map>
+                                </property>
+                                <property name="textFields">
+                                    <list>
+                                        <value>val</value>
+                                    </list>
+                                </property>
+                            </bean>
+                        </list>
+                    </property>
+                </bean>
+            </list>
+        </property>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47501</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/cache-query.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/cache-query.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/cache-query.xml
new file mode 100644
index 0000000..787a921
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/cache-query.xml
@@ -0,0 +1,100 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+
+        <property name="platformConfiguration">
+            <bean class="org.apache.ignite.platform.dotnet.PlatformDotNetConfiguration">
+                <property name="portableConfiguration">
+                    <bean class="org.apache.ignite.platform.dotnet.PlatformDotNetPortableConfiguration">
+                        <property name="types">
+                            <list>
+                                <value>Apache.Ignite.Core.Tests.Cache.Query.QueryPerson</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+
+        <property name="includeEventTypes">
+            <util:constant static-field="org.apache.ignite.events.EventType.EVTS_CACHE"/>
+        </property>
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
+
+                    <property name="typeMetadata">
+                        <list>
+                            <bean class="org.apache.ignite.cache.CacheTypeMetadata">
+                                <property name="valueType" value="QueryPerson"/>
+                                <property name="ascendingFields">
+                                    <map>
+                                        <entry key="age" value="java.lang.Integer"/>
+                                    </map>
+                                </property>
+                                <property name="queryFields">
+                                    <map>
+                                        <entry key="name" value="java.lang.String"/>
+                                        <entry key="age" value="java.lang.Integer"/>
+                                    </map>
+                                </property>
+                                <property name="textFields">
+                                    <list>
+                                        <value>name</value>
+                                    </list>
+                                </property>
+                            </bean>
+                        </list>
+                    </property>
+                </bean>
+            </list>
+        </property>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47501</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-default.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-default.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-default.xml
new file mode 100644
index 0000000..753fad1
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-default.xml
@@ -0,0 +1,43 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47502</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-invalid.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-invalid.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-invalid.xml
new file mode 100644
index 0000000..188781d
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-invalid.xml
@@ -0,0 +1,46 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+
+        <property name="marshaller">
+            <bean class="org.apache.ignite.marshaller.optimized.OptimizedMarshaller" />
+        </property>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47502</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-portable.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-portable.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-portable.xml
new file mode 100644
index 0000000..753fad1
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-portable.xml
@@ -0,0 +1,43 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47502</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache-affinity.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache-affinity.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache-affinity.xml
new file mode 100644
index 0000000..f08018d
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache-affinity.xml
@@ -0,0 +1,70 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+
+        <property name="connectorConfiguration"><null/></property>
+
+        <property name="platformConfiguration">
+            <bean class="org.apache.ignite.platform.dotnet.PlatformDotNetConfiguration">
+                <property name="portableConfiguration">
+                    <bean class="org.apache.ignite.platform.dotnet.PlatformDotNetPortableConfiguration">
+                        <property name="typesConfiguration">
+                            <list>
+                                <bean class="org.apache.ignite.platform.dotnet.PlatformDotNetPortableTypeConfiguration">
+                                    <property name="typeName"
+                                              value="Apache.Ignite.Core.Tests.Cache.CacheAffinityTest+AffinityTestKey"/>
+                                    <property name="affinityKeyFieldName" value="_affKey"/>
+                                </bean>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="cacheMode" value="PARTITIONED"/>
+                </bean>
+            </list>
+        </property>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47501</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache-parallel-store.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache-parallel-store.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache-parallel-store.xml
new file mode 100644
index 0000000..00e8e45
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache-parallel-store.xml
@@ -0,0 +1,69 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+
+        <property name="includeEventTypes">
+            <util:constant static-field="org.apache.ignite.events.EventType.EVTS_CACHE"/>
+        </property>
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="object_store_parallel"/>
+                    <property name="cacheMode" value="LOCAL"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="keepPortableInStore" value="false"/>
+
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.platform.dotnet.PlatformDotNetCacheStoreFactory">
+                            <property name="assemblyName" value="Apache.Ignite.Core.Tests"/>
+                            <property name="className" value="Apache.Ignite.Core.Tests.Cache.Store.CacheTestParallelLoadStore"/>
+                        </bean>
+                    </property>
+                </bean>
+            </list>
+        </property>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47501</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>


[04/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Portable/PortableSelfTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Portable/PortableSelfTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Portable/PortableSelfTest.cs
new file mode 100644
index 0000000..8ec6955
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Portable/PortableSelfTest.cs
@@ -0,0 +1,2078 @@
+/*
+ * 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 NonReadonlyMemberInGetHashCode
+// ReSharper disable CompareOfFloatsByEqualityOperator
+// ReSharper disable PossibleInvalidOperationException
+// ReSharper disable UnusedAutoPropertyAccessor.Global
+// ReSharper disable MemberCanBePrivate.Global
+namespace Apache.Ignite.Core.Tests.Portable 
+{
+    using System;
+    using System.Collections;
+    using System.Collections.Generic;
+    using System.Linq;
+    using System.Text;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Portable;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// 
+    /// </summary>
+    [TestFixture]
+    public class PortableSelfTest { 
+        /** */
+        private PortableMarshaller _marsh;
+
+        /// <summary>
+        /// 
+        /// </summary>
+        [TestFixtureSetUp]
+        public void BeforeTest()
+        {
+            _marsh = new PortableMarshaller(null);
+        }
+        
+        /**
+         * <summary>Check write of primitive boolean.</summary>
+         */
+        [Test]
+        public void TestWritePrimitiveBool()
+        {
+            Assert.AreEqual(_marsh.Unmarshal<bool>(_marsh.Marshal(false)), false);
+            Assert.AreEqual(_marsh.Unmarshal<bool>(_marsh.Marshal(true)), true);
+
+            Assert.AreEqual(_marsh.Unmarshal<bool?>(_marsh.Marshal((bool?)false)), false);
+            Assert.AreEqual(_marsh.Unmarshal<bool?>(_marsh.Marshal(null)), null);
+        }
+
+        /**
+         * <summary>Check write of primitive boolean array.</summary>
+         */
+        [Test]
+        public void TestWritePrimitiveBoolArray()
+        {
+            bool[] vals = { true, false };
+
+            Assert.AreEqual(_marsh.Unmarshal<bool[]>(_marsh.Marshal(vals)), vals);
+        }
+
+        /**
+         * <summary>Check write of primitive sbyte.</summary>
+         */
+        [Test]
+        public void TestWritePrimitiveSbyte()
+        {
+            Assert.AreEqual(_marsh.Unmarshal<sbyte>(_marsh.Marshal((sbyte)1)), 1);
+            Assert.AreEqual(_marsh.Unmarshal<sbyte>(_marsh.Marshal(sbyte.MinValue)), sbyte.MinValue);
+            Assert.AreEqual(_marsh.Unmarshal<sbyte>(_marsh.Marshal(sbyte.MaxValue)), sbyte.MaxValue);
+
+            Assert.AreEqual(_marsh.Unmarshal<sbyte?>(_marsh.Marshal((sbyte?)1)), (sbyte?)1);
+            Assert.AreEqual(_marsh.Unmarshal<sbyte?>(_marsh.Marshal(null)), null);
+        }
+
+        /**
+         * <summary>Check write of primitive sbyte array.</summary>
+         */
+        [Test]
+        public void TestWritePrimitiveSbyteArray()
+        {
+            sbyte[] vals = { sbyte.MinValue, 0, 1, sbyte.MaxValue };
+            sbyte[] newVals = _marsh.Unmarshal<sbyte[]>(_marsh.Marshal(vals));
+
+            Assert.AreEqual(vals, newVals);
+        }
+
+        /**
+         * <summary>Check write of primitive byte.</summary>
+         */
+        [Test]
+        public void TestWritePrimitiveByte()
+        {
+            Assert.AreEqual(_marsh.Unmarshal<byte>(_marsh.Marshal((byte)1)), 1);
+            Assert.AreEqual(_marsh.Unmarshal<byte>(_marsh.Marshal(byte.MinValue)), byte.MinValue);
+            Assert.AreEqual(_marsh.Unmarshal<byte>(_marsh.Marshal(byte.MaxValue)), byte.MaxValue);
+
+            Assert.AreEqual(_marsh.Unmarshal<byte?>(_marsh.Marshal((byte?)1)), (byte?)1);
+            Assert.AreEqual(_marsh.Unmarshal<byte?>(_marsh.Marshal(null)), null);
+        }
+
+        /**
+         * <summary>Check write of primitive byte array.</summary>
+         */
+        [Test]
+        public void TestWritePrimitiveByteArray()
+        {
+            byte[] vals = { byte.MinValue, 0, 1, byte.MaxValue };
+            byte[] newVals = _marsh.Unmarshal<byte[]>(_marsh.Marshal(vals));
+
+            Assert.AreEqual(vals, newVals);
+        }
+
+        /**
+         * <summary>Check write of primitive short.</summary>
+         */
+        [Test]
+        public void TestWritePrimitiveShort()
+        {
+            Assert.AreEqual(_marsh.Unmarshal<short>(_marsh.Marshal((short)1)), 1);
+            Assert.AreEqual(_marsh.Unmarshal<short>(_marsh.Marshal(short.MinValue)), short.MinValue);
+            Assert.AreEqual(_marsh.Unmarshal<short>(_marsh.Marshal(short.MaxValue)), short.MaxValue);
+
+            Assert.AreEqual(_marsh.Unmarshal<short?>(_marsh.Marshal((short?)1)), (short?)1);
+            Assert.AreEqual(_marsh.Unmarshal<short?>(_marsh.Marshal(null)), null);
+        }
+
+        /**
+         * <summary>Check write of primitive short array.</summary>
+         */
+        [Test]
+        public void TestWritePrimitiveShortArray()
+        {
+            short[] vals = { short.MinValue, 0, 1, short.MaxValue };
+            short[] newVals = _marsh.Unmarshal<short[]>(_marsh.Marshal(vals));
+
+            Assert.AreEqual(vals, newVals);
+        }
+
+        /**
+         * <summary>Check write of primitive ushort.</summary>
+         */
+        [Test]
+        public void TestWritePrimitiveUshort()
+        {
+            Assert.AreEqual(_marsh.Unmarshal<ushort>(_marsh.Marshal((ushort)1)), 1);
+            Assert.AreEqual(_marsh.Unmarshal<ushort>(_marsh.Marshal(ushort.MinValue)), ushort.MinValue);
+            Assert.AreEqual(_marsh.Unmarshal<ushort>(_marsh.Marshal(ushort.MaxValue)), ushort.MaxValue);
+
+            Assert.AreEqual(_marsh.Unmarshal<ushort?>(_marsh.Marshal((ushort?)1)), (ushort?)1);
+            Assert.AreEqual(_marsh.Unmarshal<ushort?>(_marsh.Marshal(null)), null);
+        }
+
+        /**
+         * <summary>Check write of primitive short array.</summary>
+         */
+        [Test]
+        public void TestWritePrimitiveUshortArray()
+        {
+            ushort[] vals = { ushort.MinValue, 0, 1, ushort.MaxValue };
+            ushort[] newVals = _marsh.Unmarshal<ushort[]>(_marsh.Marshal(vals));
+
+            Assert.AreEqual(vals, newVals);
+        }
+
+        /**
+         * <summary>Check write of primitive char.</summary>
+         */
+        [Test]
+        public void TestWritePrimitiveChar()
+        {
+            Assert.AreEqual(_marsh.Unmarshal<char>(_marsh.Marshal((char)1)), (char)1);
+            Assert.AreEqual(_marsh.Unmarshal<char>(_marsh.Marshal(char.MinValue)), char.MinValue);
+            Assert.AreEqual(_marsh.Unmarshal<char>(_marsh.Marshal(char.MaxValue)), char.MaxValue);
+
+            Assert.AreEqual(_marsh.Unmarshal<char?>(_marsh.Marshal((char?)1)), (char?)1);
+            Assert.AreEqual(_marsh.Unmarshal<char?>(_marsh.Marshal(null)), null);
+        }
+
+        /**
+         * <summary>Check write of primitive uint array.</summary>
+         */
+        [Test]
+        public void TestWritePrimitiveCharArray()
+        {
+            char[] vals = { char.MinValue, (char)0, (char)1, char.MaxValue };
+            char[] newVals = _marsh.Unmarshal<char[]>(_marsh.Marshal(vals));
+
+            Assert.AreEqual(vals, newVals);
+        }
+
+        /**
+         * <summary>Check write of primitive int.</summary>
+         */
+        [Test]
+        public void TestWritePrimitiveInt()
+        {
+            Assert.AreEqual(_marsh.Unmarshal<int>(_marsh.Marshal(1)), 1);
+            Assert.AreEqual(_marsh.Unmarshal<int>(_marsh.Marshal(int.MinValue)), int.MinValue);
+            Assert.AreEqual(_marsh.Unmarshal<int>(_marsh.Marshal(int.MaxValue)), int.MaxValue);
+
+            Assert.AreEqual(_marsh.Unmarshal<int?>(_marsh.Marshal((int?)1)), (int?)1);
+            Assert.AreEqual(_marsh.Unmarshal<int?>(_marsh.Marshal(null)), null);
+        }
+
+        /**
+         * <summary>Check write of primitive uint array.</summary>
+         */
+        [Test]
+        public void TestWritePrimitiveIntArray()
+        {
+            int[] vals = { int.MinValue, 0, 1, int.MaxValue };
+            int[] newVals = _marsh.Unmarshal<int[]>(_marsh.Marshal(vals));
+
+            Assert.AreEqual(vals, newVals);
+        }
+
+        /**
+         * <summary>Check write of primitive uint.</summary>
+         */
+        [Test]
+        public void TestWritePrimitiveUint()
+        {
+            Assert.AreEqual(_marsh.Unmarshal<uint>(_marsh.Marshal((uint)1)), 1);
+            Assert.AreEqual(_marsh.Unmarshal<uint>(_marsh.Marshal(uint.MinValue)), uint.MinValue);
+            Assert.AreEqual(_marsh.Unmarshal<uint>(_marsh.Marshal(uint.MaxValue)), uint.MaxValue);
+
+            Assert.AreEqual(_marsh.Unmarshal<uint?>(_marsh.Marshal((uint?)1)), (int?)1);
+            Assert.AreEqual(_marsh.Unmarshal<uint?>(_marsh.Marshal(null)), null);
+        }
+
+        /**
+         * <summary>Check write of primitive uint array.</summary>
+         */
+        [Test]
+        public void TestWritePrimitiveUintArray()
+        {
+            uint[] vals = { uint.MinValue, 0, 1, uint.MaxValue };
+            uint[] newVals = _marsh.Unmarshal<uint[]>(_marsh.Marshal(vals));
+
+            Assert.AreEqual(vals, newVals);
+        }
+
+        /**
+         * <summary>Check write of primitive long.</summary>
+         */
+        [Test]
+        public void TestWritePrimitiveLong()
+        {
+            Assert.AreEqual(_marsh.Unmarshal<long>(_marsh.Marshal((long)1)), 1);
+            Assert.AreEqual(_marsh.Unmarshal<long>(_marsh.Marshal(long.MinValue)), long.MinValue);
+            Assert.AreEqual(_marsh.Unmarshal<long>(_marsh.Marshal(long.MaxValue)), long.MaxValue);
+
+            Assert.AreEqual(_marsh.Unmarshal<long?>(_marsh.Marshal((long?)1)), (long?)1);
+            Assert.AreEqual(_marsh.Unmarshal<long?>(_marsh.Marshal(null)), null);
+        }
+
+        /**
+         * <summary>Check write of primitive long array.</summary>
+         */
+        [Test]
+        public void TestWritePrimitiveLongArray()
+        {
+            long[] vals = { long.MinValue, 0, 1, long.MaxValue };
+            long[] newVals = _marsh.Unmarshal<long[]>(_marsh.Marshal(vals));
+
+            Assert.AreEqual(vals, newVals);
+        }
+
+        /**
+         * <summary>Check write of primitive ulong.</summary>
+         */
+        [Test]
+        public void TestWritePrimitiveUlong()
+        {
+            Assert.AreEqual(_marsh.Unmarshal<ulong>(_marsh.Marshal((ulong)1)), 1);
+            Assert.AreEqual(_marsh.Unmarshal<ulong>(_marsh.Marshal(ulong.MinValue)), ulong.MinValue);
+            Assert.AreEqual(_marsh.Unmarshal<ulong>(_marsh.Marshal(ulong.MaxValue)), ulong.MaxValue);
+
+            Assert.AreEqual(_marsh.Unmarshal<ulong?>(_marsh.Marshal((ulong?)1)), (ulong?)1);
+            Assert.AreEqual(_marsh.Unmarshal<ulong?>(_marsh.Marshal(null)), null);
+        }
+
+        /**
+         * <summary>Check write of primitive ulong array.</summary>
+         */
+        [Test]
+        public void TestWritePrimitiveUlongArray()
+        {
+            ulong[] vals = { ulong.MinValue, 0, 1, ulong.MaxValue };
+            ulong[] newVals = _marsh.Unmarshal<ulong[]>(_marsh.Marshal(vals));
+
+            Assert.AreEqual(vals, newVals);
+        }
+
+        /**
+         * <summary>Check write of primitive float.</summary>
+         */
+        [Test]
+        public void TestWritePrimitiveFloat()
+        {
+            Assert.AreEqual(_marsh.Unmarshal<float>(_marsh.Marshal((float)1)), (float)1);
+            Assert.AreEqual(_marsh.Unmarshal<float>(_marsh.Marshal(float.MinValue)), float.MinValue);
+            Assert.AreEqual(_marsh.Unmarshal<float>(_marsh.Marshal(float.MaxValue)), float.MaxValue);
+
+            Assert.AreEqual(_marsh.Unmarshal<float?>(_marsh.Marshal((float?)1)), (float?)1);
+            Assert.AreEqual(_marsh.Unmarshal<float?>(_marsh.Marshal(null)), null);
+        }
+
+        /**
+         * <summary>Check write of primitive float array.</summary>
+         */
+        [Test]
+        public void TestWritePrimitiveFloatArray()
+        {
+            float[] vals = { float.MinValue, 0, 1, float.MaxValue };
+            float[] newVals = _marsh.Unmarshal<float[]>(_marsh.Marshal(vals));
+
+            Assert.AreEqual(vals, newVals);
+        }
+
+        /**
+         * <summary>Check write of primitive double.</summary>
+         */
+        [Test]
+        public void TestWritePrimitiveDouble()
+        {
+            Assert.AreEqual(_marsh.Unmarshal<double>(_marsh.Marshal((double)1)), (double)1);
+            Assert.AreEqual(_marsh.Unmarshal<double>(_marsh.Marshal(double.MinValue)), double.MinValue);
+            Assert.AreEqual(_marsh.Unmarshal<double>(_marsh.Marshal(double.MaxValue)), double.MaxValue);
+
+            Assert.AreEqual(_marsh.Unmarshal<double?>(_marsh.Marshal((double?)1)), (double?)1);
+            Assert.AreEqual(_marsh.Unmarshal<double?>(_marsh.Marshal(null)), null);
+        }
+
+        /**
+         * <summary>Check write of primitive double array.</summary>
+         */
+        [Test]
+        public void TestWritePrimitiveDoubleArray()
+        {
+            double[] vals = { double.MinValue, 0, 1, double.MaxValue };
+            double[] newVals = _marsh.Unmarshal<double[]>(_marsh.Marshal(vals));
+
+            Assert.AreEqual(vals, newVals);
+        }
+
+        /**
+         * <summary>Check write of decimal.</summary>
+         */
+        [Test]
+        public void TestWritePrimitiveDecimal()
+        {
+            decimal val;
+
+            // Test positibe and negative.
+            Assert.AreEqual(_marsh.Unmarshal<decimal>(_marsh.Marshal(val = decimal.Zero)), val);
+            Assert.AreEqual(_marsh.Unmarshal<decimal>(_marsh.Marshal(val = new decimal(1, 0, 0, false, 0))), val);
+            Assert.AreEqual(_marsh.Unmarshal<decimal>(_marsh.Marshal(val = new decimal(1, 0, 0, true, 0))), val);
+
+            // Test 32, 64 and 96 bits + mixed.
+            Assert.AreEqual(_marsh.Unmarshal<decimal>(_marsh.Marshal(val = new decimal(0, 1, 0, false, 0))), val);
+            Assert.AreEqual(_marsh.Unmarshal<decimal>(_marsh.Marshal(val = new decimal(0, 1, 0, true, 0))), val);
+            Assert.AreEqual(_marsh.Unmarshal<decimal>(_marsh.Marshal(val = new decimal(0, 0, 1, false, 0))), val);
+            Assert.AreEqual(_marsh.Unmarshal<decimal>(_marsh.Marshal(val = new decimal(0, 0, 1, true, 0))), val);
+            Assert.AreEqual(_marsh.Unmarshal<decimal>(_marsh.Marshal(val = new decimal(1, 1, 1, false, 0))), val);
+            Assert.AreEqual(_marsh.Unmarshal<decimal>(_marsh.Marshal(val = new decimal(1, 1, 1, true, 0))), val);
+
+            // Test extremes.
+            Assert.AreEqual(_marsh.Unmarshal<decimal>(_marsh.Marshal(val = decimal.Parse("65536"))), val);
+            Assert.AreEqual(_marsh.Unmarshal<decimal>(_marsh.Marshal(val = decimal.Parse("-65536"))), val);
+
+            Assert.AreEqual(_marsh.Unmarshal<decimal>(_marsh.Marshal(val = decimal.Parse("4294967296"))), val);
+            Assert.AreEqual(_marsh.Unmarshal<decimal>(_marsh.Marshal(val = decimal.Parse("-4294967296"))), val);
+
+            Assert.AreEqual(_marsh.Unmarshal<decimal>(_marsh.Marshal(val = decimal.Parse("281474976710656"))), val);
+            Assert.AreEqual(_marsh.Unmarshal<decimal>(_marsh.Marshal(val = decimal.Parse("-281474976710656"))), val);
+
+            Assert.AreEqual(_marsh.Unmarshal<decimal>(_marsh.Marshal(val = decimal.Parse("18446744073709551616"))), val);
+            Assert.AreEqual(_marsh.Unmarshal<decimal>(_marsh.Marshal(val = decimal.Parse("-18446744073709551616"))), val);
+
+            Assert.AreEqual(_marsh.Unmarshal<decimal>(_marsh.Marshal(val = decimal.Parse("1208925819614629174706176"))), val);
+            Assert.AreEqual(_marsh.Unmarshal<decimal>(_marsh.Marshal(val = decimal.Parse("-1208925819614629174706176"))), val);
+
+            Assert.AreEqual(_marsh.Unmarshal<decimal>(_marsh.Marshal(val = decimal.MaxValue)), val);
+            Assert.AreEqual(_marsh.Unmarshal<decimal>(_marsh.Marshal(val = decimal.MinValue)), val);
+
+            // Test scale.
+            Assert.AreEqual(_marsh.Unmarshal<decimal>(_marsh.Marshal(val = decimal.Parse("11,12"))), val);
+            Assert.AreEqual(_marsh.Unmarshal<decimal>(_marsh.Marshal(val = decimal.Parse("-11,12"))), val);
+
+            // Test null.
+            Assert.AreEqual(_marsh.Unmarshal<decimal?>(_marsh.Marshal(null)), null);
+        }
+
+        /**
+         * <summary>Check write of decimal array.</summary>
+         */
+        [Test]
+        public void TestWritePrimitiveDecimalArray()
+        {
+            decimal[] vals = { decimal.One, decimal.Parse("11,12") };
+            decimal[] newVals = _marsh.Unmarshal<decimal[]>(_marsh.Marshal(vals));
+
+            Assert.AreEqual(vals, newVals);
+        }
+
+        /**
+         * <summary>Check write of string.</summary>
+         */
+        [Test]
+        public void TestWriteString()
+        {
+            Assert.AreEqual(_marsh.Unmarshal<string>(_marsh.Marshal("str")), "str");
+            Assert.AreEqual(_marsh.Unmarshal<string>(_marsh.Marshal(null)), null);
+        }
+
+        /**
+         * <summary>Check write of string array.</summary>
+         */
+        [Test]
+        public void TestWriteStringArray()
+        {
+            string[] vals = { "str1", null, "", "str2", null};
+            string[] newVals = _marsh.Unmarshal<string[]>(_marsh.Marshal(vals));
+
+            Assert.AreEqual(vals, newVals);
+        }
+
+        /**
+         * <summary>Check write of Guid.</summary>
+         */
+        [Test]
+        public void TestWriteGuid()
+        {
+            Guid guid = Guid.NewGuid();
+            Guid? nGuid = guid;
+
+            Assert.AreEqual(_marsh.Unmarshal<Guid>(_marsh.Marshal(guid)), guid);
+            Assert.AreEqual(_marsh.Unmarshal<Guid?>(_marsh.Marshal(nGuid)), nGuid);
+
+            nGuid = null;
+
+            // ReSharper disable once ExpressionIsAlwaysNull
+            Assert.AreEqual(_marsh.Unmarshal<Guid?>(_marsh.Marshal(nGuid)), null);
+        }
+
+        /**
+         * <summary>Check write of string array.</summary>
+         */
+        [Test]
+        public void TestWriteGuidArray()
+        {
+            Guid?[] vals = { Guid.NewGuid(), null, Guid.Empty, Guid.NewGuid(), null };
+            Guid?[] newVals = _marsh.Unmarshal<Guid?[]>(_marsh.Marshal(vals));
+
+            Assert.AreEqual(vals, newVals);
+        }
+
+        /**
+        * <summary>Check write of enum.</summary>
+        */
+        [Test]
+        public void TestWriteEnum()
+        {
+            TestEnum val = TestEnum.Val1;
+
+            Assert.AreEqual(_marsh.Unmarshal<TestEnum>(_marsh.Marshal(val)), val);
+        }
+
+        /**
+        * <summary>Check write of enum.</summary>
+        */
+        [Test]
+        public void TestWriteEnumArray()
+        {
+            TestEnum[] vals = { TestEnum.Val2, TestEnum.Val3 };
+            TestEnum[] newVals = _marsh.Unmarshal<TestEnum[]>(_marsh.Marshal(vals));
+
+            Assert.AreEqual(vals, newVals);
+        }
+
+        /**
+         * <summary>Check write of date.</summary>
+         */
+        [Test]
+        public void TestWriteDate() {
+            DateTime time = DateTime.Now.ToUniversalTime();
+
+            Assert.AreEqual(_marsh.Unmarshal<DateTime>(_marsh.Marshal(time)), time);
+        }
+
+        /// <summary>
+        /// Test object with dates.
+        /// </summary>
+        [Test]
+        public void TestDateObject()
+        {
+            ICollection<PortableTypeConfiguration> typeCfgs =
+                new List<PortableTypeConfiguration>();
+
+            typeCfgs.Add(new PortableTypeConfiguration(typeof(DateTimeType)));
+
+            PortableConfiguration cfg = new PortableConfiguration {TypeConfigurations = typeCfgs};
+
+            PortableMarshaller marsh = new PortableMarshaller(cfg);
+
+            DateTime now = DateTime.Now;
+
+            DateTimeType obj = new DateTimeType(now);
+
+            DateTimeType otherObj = marsh.Unmarshal<DateTimeType>(marsh.Marshal(obj));
+
+            Assert.AreEqual(obj.Loc, otherObj.Loc);
+            Assert.AreEqual(obj.Utc, otherObj.Utc);
+            Assert.AreEqual(obj.LocNull, otherObj.LocNull);
+            Assert.AreEqual(obj.UtcNull, otherObj.UtcNull);            
+            Assert.AreEqual(obj.LocArr, otherObj.LocArr);
+            Assert.AreEqual(obj.UtcArr, otherObj.UtcArr);
+
+            Assert.AreEqual(obj.LocRaw, otherObj.LocRaw);
+            Assert.AreEqual(obj.UtcRaw, otherObj.UtcRaw);
+            Assert.AreEqual(obj.LocNullRaw, otherObj.LocNullRaw);
+            Assert.AreEqual(obj.UtcNullRaw, otherObj.UtcNullRaw);
+            Assert.AreEqual(obj.LocArrRaw, otherObj.LocArrRaw);
+            Assert.AreEqual(obj.UtcArrRaw, otherObj.UtcArrRaw);
+        }
+
+        /**
+         * <summary>Check generic collections.</summary>
+         */
+        [Test]
+        public void TestGenericCollections()
+        {
+            ICollection<string> list = new List<string>();
+
+            list.Add("1");
+
+            byte[] data = _marsh.Marshal(list);
+
+            ICollection<object> newObjList = _marsh.Unmarshal<List<object>>(data);
+
+            Assert.NotNull(newObjList);
+
+            ICollection<string> newList = new List<string>();
+
+            foreach (object obj in newObjList)
+                newList.Add((string)obj);
+
+            CollectionAssert.AreEquivalent(list, newList);
+        }
+
+        /**
+         * <summary>Check property read.</summary>
+         */
+        [Test]
+        public void TestProperty()
+        {
+            ICollection<PortableTypeConfiguration> typeCfgs = 
+                new List<PortableTypeConfiguration>();
+
+            typeCfgs.Add(new PortableTypeConfiguration(typeof(PropertyType)));
+
+            PortableConfiguration cfg = new PortableConfiguration {TypeConfigurations = typeCfgs};
+
+            PortableMarshaller marsh = new PortableMarshaller(cfg);
+
+            PropertyType obj = new PropertyType
+            {
+                Field1 = 1,
+                Field2 = 2
+            };
+
+            byte[] data = marsh.Marshal(obj);
+
+            PropertyType newObj = marsh.Unmarshal<PropertyType>(data);
+
+            Assert.AreEqual(obj.Field1, newObj.Field1);
+            Assert.AreEqual(obj.Field2, newObj.Field2);
+
+            IPortableObject portNewObj = marsh.Unmarshal<IPortableObject>(data, PortableMode.ForcePortable);
+
+            Assert.AreEqual(obj.Field1, portNewObj.Field<int>("field1"));
+            Assert.AreEqual(obj.Field2, portNewObj.Field<int>("Field2"));
+        }
+
+        /**
+         * <summary>Check write of primitive fields through reflection.</summary>
+         */
+        [Test]
+        public void TestPrimitiveFieldsReflective()
+        {
+            ICollection<PortableTypeConfiguration> typeCfgs = 
+                new List<PortableTypeConfiguration>();
+
+            typeCfgs.Add(new PortableTypeConfiguration(typeof(PrimitiveFieldType)));
+
+            PortableConfiguration cfg = new PortableConfiguration {TypeConfigurations = typeCfgs};
+
+            PortableMarshaller marsh = new PortableMarshaller(cfg);
+
+            PrimitiveFieldType obj = new PrimitiveFieldType();
+
+            CheckPrimitiveFields(marsh, obj);
+        }
+
+        /**
+         * <summary>Check write of primitive fields through portable interface.</summary>
+         */
+        [Test]
+        public void TestPrimitiveFieldsPortable()
+        {
+            ICollection<PortableTypeConfiguration> typeCfgs = 
+                new List<PortableTypeConfiguration>();
+
+            typeCfgs.Add(new PortableTypeConfiguration(typeof(PrimitiveFieldPortableType)));
+
+            PortableConfiguration cfg = new PortableConfiguration();
+
+            cfg.TypeConfigurations = typeCfgs;
+
+            PortableMarshaller marsh = new PortableMarshaller(cfg);
+
+            PrimitiveFieldPortableType obj = new PrimitiveFieldPortableType();
+
+            CheckPrimitiveFields(marsh, obj);
+        }
+
+        /**
+         * <summary>Check write of primitive fields through portable interface.</summary>
+         */
+        [Test]
+        public void TestPrimitiveFieldsRawPortable()
+        {
+            ICollection<PortableTypeConfiguration> typeCfgs = 
+                new List<PortableTypeConfiguration>();
+
+            typeCfgs.Add(new PortableTypeConfiguration(typeof(PrimitiveFieldRawPortableType)));
+
+            PortableConfiguration cfg = new PortableConfiguration();
+
+            cfg.TypeConfigurations = typeCfgs;
+
+            PortableMarshaller marsh = new PortableMarshaller(cfg);
+
+            PrimitiveFieldRawPortableType obj = new PrimitiveFieldRawPortableType();
+
+            CheckPrimitiveFields(marsh, obj);
+        }
+
+        /**
+         * <summary>Check write of primitive fields through portable interface.</summary>
+         */
+        [Test]
+        public void TestPrimitiveFieldsSerializer()
+        {
+            var typeCfgs = new List<PortableTypeConfiguration>
+            {
+                new PortableTypeConfiguration(typeof (PrimitiveFieldType))
+                {
+                    Serializer = new PrimitiveFieldsSerializer()
+                }
+            };
+
+            PortableConfiguration cfg = new PortableConfiguration {TypeConfigurations = typeCfgs};
+
+            PortableMarshaller marsh = new PortableMarshaller(cfg);
+
+            PrimitiveFieldType obj = new PrimitiveFieldType();
+
+            CheckPrimitiveFields(marsh, obj);
+        }
+
+        /**
+         * <summary>Check decimals.</summary>
+         */
+        [Test]
+        public void TestDecimalFields()
+        {
+            PortableConfiguration cfg = new PortableConfiguration
+            {
+                TypeConfigurations = new List<PortableTypeConfiguration>
+                {
+                    new PortableTypeConfiguration(typeof (DecimalReflective)),
+                    new PortableTypeConfiguration(typeof (DecimalMarshalAware))
+                }
+            };
+
+            PortableMarshaller marsh = new PortableMarshaller(cfg);
+
+            // 1. Test reflective stuff.
+            DecimalReflective obj1 = new DecimalReflective
+            {
+                Val = decimal.Zero,
+                ValArr = new[] {decimal.One, decimal.MinusOne}
+            };
+
+            IPortableObject portObj = marsh.Unmarshal<IPortableObject>(marsh.Marshal(obj1), PortableMode.ForcePortable);
+
+            Assert.AreEqual(obj1.Val, portObj.Field<decimal>("val"));
+            Assert.AreEqual(obj1.ValArr, portObj.Field<decimal[]>("valArr"));
+
+            Assert.AreEqual(obj1.Val, portObj.Deserialize<DecimalReflective>().Val);
+            Assert.AreEqual(obj1.ValArr, portObj.Deserialize<DecimalReflective>().ValArr);
+
+            // 2. Test marshal aware stuff.
+            DecimalMarshalAware obj2 = new DecimalMarshalAware();
+
+            obj2.Val = decimal.Zero;
+            obj2.ValArr = new[] { decimal.One, decimal.MinusOne };
+            obj2.RawVal = decimal.MaxValue;
+            obj2.RawValArr = new[] { decimal.MinusOne, decimal.One} ;
+
+            portObj = marsh.Unmarshal<IPortableObject>(marsh.Marshal(obj2), PortableMode.ForcePortable);
+
+            Assert.AreEqual(obj2.Val, portObj.Field<decimal>("val"));
+            Assert.AreEqual(obj2.ValArr, portObj.Field<decimal[]>("valArr"));
+
+            Assert.AreEqual(obj2.Val, portObj.Deserialize<DecimalMarshalAware>().Val);
+            Assert.AreEqual(obj2.ValArr, portObj.Deserialize<DecimalMarshalAware>().ValArr);
+            Assert.AreEqual(obj2.RawVal, portObj.Deserialize<DecimalMarshalAware>().RawVal);
+            Assert.AreEqual(obj2.RawValArr, portObj.Deserialize<DecimalMarshalAware>().RawValArr);
+        }
+
+        /**
+         * <summary>Check write of primitive fields through raw serializer.</summary>
+         */
+        [Test]
+        public void TestPrimitiveFieldsRawSerializer()
+        {
+            ICollection<PortableTypeConfiguration> typeCfgs = 
+                new List<PortableTypeConfiguration>();
+
+            PortableTypeConfiguration typeCfg =
+                new PortableTypeConfiguration(typeof(PrimitiveFieldType));
+
+            typeCfg.Serializer = new PrimitiveFieldsRawSerializer();
+
+            typeCfgs.Add(typeCfg);
+
+            PortableConfiguration cfg = new PortableConfiguration();
+
+            cfg.TypeConfigurations = typeCfgs;
+
+            PortableMarshaller marsh = new PortableMarshaller(cfg);
+
+            PrimitiveFieldType obj = new PrimitiveFieldType();
+
+            CheckPrimitiveFields(marsh, obj);
+        }
+
+        private void CheckPrimitiveFields(PortableMarshaller marsh, PrimitiveFieldType obj)
+        {
+            obj.PBool = true;
+            obj.PByte = 2;
+            obj.PSbyte = 3;
+            obj.PShort = 4;
+            obj.PUshort = 5;
+            obj.PInt = 6;
+            obj.PUint = 7;
+            obj.PLong = 8;
+            obj.PUlong = 9;
+            obj.PChar = 'a';
+            obj.PFloat = 10;
+            obj.PDouble = 11;
+            obj.PString = "abc";
+            obj.PGuid = Guid.NewGuid();
+            obj.PnGuid = Guid.NewGuid();
+            
+            //CheckPrimitiveFieldsSerialization(marsh, obj);
+
+            //obj.PString = "";
+
+            //CheckPrimitiveFieldsSerialization(marsh, obj);
+
+            //obj.PString = null;
+
+            //CheckPrimitiveFieldsSerialization(marsh, obj);
+
+            //obj.PString = null;
+            //obj.PNGuid = null;
+
+            CheckPrimitiveFieldsSerialization(marsh, obj);
+        }
+
+        private void CheckPrimitiveFieldsSerialization(PortableMarshaller marsh, PrimitiveFieldType obj)
+        {
+            byte[] bytes = marsh.Marshal(obj);
+
+            IPortableObject portObj = marsh.Unmarshal<IPortableObject>(bytes, PortableMode.ForcePortable);
+
+            Assert.AreEqual(obj.GetHashCode(), portObj.GetHashCode());
+
+            PrimitiveFieldType newObj = portObj.Deserialize<PrimitiveFieldType>();
+
+            Assert.AreEqual(obj, newObj);
+        }
+
+        /**
+         * <summary>Check write of object with enums.</summary>
+         */
+        [Test]
+        public void TestEnumsReflective()
+        {
+            PortableMarshaller marsh =
+                new PortableMarshaller(new PortableConfiguration
+                {
+                    TypeConfigurations =
+                        new List<PortableTypeConfiguration> {new PortableTypeConfiguration(typeof (EnumType))}
+                });
+
+            EnumType obj = new EnumType
+            {
+                PEnum = TestEnum.Val1,
+                PEnumArray = new[] {TestEnum.Val2, TestEnum.Val3}
+            };
+
+            byte[] bytes = marsh.Marshal(obj);
+
+            IPortableObject portObj = marsh.Unmarshal<IPortableObject>(bytes, PortableMode.ForcePortable);
+
+            Assert.AreEqual(obj.GetHashCode(), portObj.GetHashCode());
+
+            EnumType newObj = portObj.Deserialize<EnumType>();
+
+            Assert.AreEqual(obj.PEnum, newObj.PEnum);
+            Assert.AreEqual(obj.PEnumArray, newObj.PEnumArray);
+        }
+
+        /**
+         * <summary>Check write of object with collections.</summary>
+         */
+        [Test]
+        public void TestCollectionsReflective()
+        {
+            ICollection<PortableTypeConfiguration> typeCfgs =
+                new List<PortableTypeConfiguration>();
+
+            typeCfgs.Add(new PortableTypeConfiguration(typeof(CollectionsType)));
+            typeCfgs.Add(new PortableTypeConfiguration(typeof(InnerObjectType)));
+
+            PortableConfiguration cfg = new PortableConfiguration();
+
+            cfg.TypeConfigurations = typeCfgs;
+
+            PortableMarshaller marsh = new PortableMarshaller(cfg);
+
+            CollectionsType obj = new CollectionsType();
+
+            ArrayList list = new ArrayList();
+
+            list.Add(true);
+            list.Add((byte)1);
+            list.Add((short)2);
+            list.Add('a');
+            list.Add(3);
+            list.Add((long)4);
+            list.Add((float)5);
+            list.Add((double)6);
+
+            list.Add("string");
+            list.Add(Guid.NewGuid());
+
+            InnerObjectType innerObj = new InnerObjectType();
+
+            innerObj.PInt1 = 1;
+            innerObj.PInt2 = 2;
+            
+            list.Add(innerObj);
+
+            obj.Col1 = list;
+
+            byte[] bytes = marsh.Marshal(obj);
+
+            IPortableObject portObj = marsh.Unmarshal<IPortableObject>(bytes, PortableMode.ForcePortable);
+
+            Assert.AreEqual(obj.GetHashCode(), portObj.GetHashCode());
+
+            CollectionsType newObj = portObj.Deserialize<CollectionsType>();
+
+            Assert.AreEqual(obj, newObj);
+
+            obj.Col1 = null;
+
+            Assert.AreEqual(obj, marsh.Unmarshal<CollectionsType>(marsh.Marshal(obj)));
+
+            obj.Col1 = list;
+            obj.Col2 = list;
+
+            Assert.AreEqual(obj, marsh.Unmarshal<CollectionsType>(marsh.Marshal(obj)));
+
+            obj.Col2 = new TestList();
+
+            Assert.AreEqual(obj, marsh.Unmarshal<CollectionsType>(marsh.Marshal(obj)));
+        }
+
+        /**
+         * <summary>Check write of object fields through reflective serializer.</summary>
+         */
+        [Test]
+        public void TestObjectReflective()
+        {
+            ICollection<PortableTypeConfiguration> typeCfgs = 
+                new List<PortableTypeConfiguration>();
+
+            typeCfgs.Add(new PortableTypeConfiguration(typeof(OuterObjectType)));
+            typeCfgs.Add(new PortableTypeConfiguration(typeof(InnerObjectType)));
+
+            PortableConfiguration cfg = new PortableConfiguration();
+
+            cfg.TypeConfigurations = typeCfgs;
+
+            PortableMarshaller marsh = new PortableMarshaller(cfg);
+
+            CheckObject(marsh, new OuterObjectType(), new InnerObjectType());
+        }
+
+        /**
+         * <summary>Test handles.</summary>
+         */
+        [Test]
+        public void TestHandles()
+        {
+            ICollection<PortableTypeConfiguration> typeCfgs =
+                new List<PortableTypeConfiguration>();
+
+            typeCfgs.Add(new PortableTypeConfiguration(typeof(HandleInner)));
+            typeCfgs.Add(new PortableTypeConfiguration(typeof(HandleOuter)));
+
+            PortableConfiguration cfg = new PortableConfiguration();
+
+            cfg.TypeConfigurations = typeCfgs;
+
+            PortableMarshaller marsh = new PortableMarshaller(cfg);
+
+            HandleOuter outer = new HandleOuter();
+
+            outer.Before = "outBefore";
+            outer.After = "outAfter";
+            outer.RawBefore = "outRawBefore";
+            outer.RawAfter = "outRawAfter";
+
+            HandleInner inner = new HandleInner();
+
+            inner.Before = "inBefore";
+            inner.After = "inAfter";
+            inner.RawBefore = "inRawBefore";
+            inner.RawAfter = "inRawAfter";
+
+            outer.Inner = inner;
+            outer.RawInner = inner;
+
+            inner.Outer = outer;
+            inner.RawOuter = outer;
+
+            byte[] bytes = marsh.Marshal(outer);
+
+            IPortableObject outerObj = marsh.Unmarshal<IPortableObject>(bytes, PortableMode.ForcePortable);
+
+            HandleOuter newOuter = outerObj.Deserialize<HandleOuter>();
+            HandleInner newInner = newOuter.Inner;
+
+            CheckHandlesConsistency(outer, inner, newOuter, newInner);
+
+            // Get inner object by field.
+            IPortableObject innerObj = outerObj.Field<IPortableObject>("inner");
+
+            newInner = innerObj.Deserialize<HandleInner>();
+            newOuter = newInner.Outer;
+
+            CheckHandlesConsistency(outer, inner, newOuter, newInner);
+
+            // Get outer object from inner object by handle.
+            outerObj = innerObj.Field<IPortableObject>("outer");
+
+            newOuter = outerObj.Deserialize<HandleOuter>();
+            newInner = newOuter.Inner;
+
+            CheckHandlesConsistency(outer, inner, newOuter, newInner);
+        }
+
+        /**
+         * <summary>Test handles with exclusive writes.</summary>
+         */
+        [Test]
+        public void TestHandlesExclusive([Values(true, false)] bool detached, [Values(true, false)] bool asPortable)
+        {
+            var marsh = new PortableMarshaller(new PortableConfiguration
+            {
+                TypeConfigurations = new List<PortableTypeConfiguration>
+                {
+                    new PortableTypeConfiguration(typeof (HandleInner)),
+                    new PortableTypeConfiguration(typeof (HandleOuterExclusive))
+                }
+            });
+
+            var inner = new HandleInner
+            {
+                Before = "inBefore",
+                After = "inAfter",
+                RawBefore = "inRawBefore",
+                RawAfter = "inRawAfter"
+            };
+
+            var outer = new HandleOuterExclusive
+            {
+                Before = "outBefore",
+                After = "outAfter",
+                RawBefore = "outRawBefore",
+                RawAfter = "outRawAfter",
+                Inner = inner,
+                RawInner = inner
+            };
+
+            inner.Outer = outer;
+            inner.RawOuter = outer;
+
+            var bytes = asPortable
+                ? marsh.Marshal(new PortablesImpl(marsh).ToPortable<IPortableObject>(outer))
+                : marsh.Marshal(outer);
+
+            IPortableObject outerObj;
+
+            if (detached)
+            {
+                var reader = new PortableReaderImpl(marsh, new Dictionary<long, IPortableTypeDescriptor>(),
+                    new PortableHeapStream(bytes), PortableMode.ForcePortable, null);
+
+                reader.DetachNext();
+
+                outerObj = reader.Deserialize<IPortableObject>();
+            }
+            else
+                outerObj = marsh.Unmarshal<IPortableObject>(bytes, PortableMode.ForcePortable);
+
+            HandleOuter newOuter = outerObj.Deserialize<HandleOuter>();
+
+            Assert.IsFalse(newOuter == newOuter.Inner.Outer);
+            Assert.IsFalse(newOuter == newOuter.Inner.RawOuter);
+            Assert.IsFalse(newOuter == newOuter.RawInner.RawOuter);
+            Assert.IsFalse(newOuter == newOuter.RawInner.RawOuter);
+
+            Assert.IsFalse(newOuter.Inner == newOuter.RawInner);
+
+            Assert.IsTrue(newOuter.Inner.Outer == newOuter.Inner.RawOuter);
+            Assert.IsTrue(newOuter.RawInner.Outer == newOuter.RawInner.RawOuter);
+
+            Assert.IsTrue(newOuter.Inner == newOuter.Inner.Outer.Inner);
+            Assert.IsTrue(newOuter.Inner == newOuter.Inner.Outer.RawInner);
+            Assert.IsTrue(newOuter.RawInner == newOuter.RawInner.Outer.Inner);
+            Assert.IsTrue(newOuter.RawInner == newOuter.RawInner.Outer.RawInner);
+        }
+
+        ///
+        /// <summary>Test KeepSerialized property</summary>
+        ///
+        [Test]
+        public void TestKeepSerializedDefault()
+        {
+            CheckKeepSerialized(new PortableConfiguration(), true);
+        }
+
+        ///
+        /// <summary>Test KeepSerialized property</summary>
+        ///
+        [Test]
+        public void TestKeepSerializedDefaultFalse()
+        {
+            PortableConfiguration cfg = new PortableConfiguration();
+
+            cfg.DefaultKeepDeserialized = false;
+
+            CheckKeepSerialized(cfg, false);
+        }
+
+        ///
+        /// <summary>Test KeepSerialized property</summary>
+        ///
+        [Test]
+        public void TestKeepSerializedTypeCfgFalse()
+        {
+            PortableTypeConfiguration typeCfg = new PortableTypeConfiguration(typeof(PropertyType));
+
+            typeCfg.KeepDeserialized = false;
+
+            PortableConfiguration cfg = new PortableConfiguration();
+
+            cfg.TypeConfigurations = new List<PortableTypeConfiguration> { typeCfg };
+
+            CheckKeepSerialized(cfg, false);
+        }
+
+        ///
+        /// <summary>Test KeepSerialized property</summary>
+        ///
+        [Test]
+        public void TestKeepSerializedTypeCfgTrue()
+        {
+            PortableTypeConfiguration typeCfg = new PortableTypeConfiguration(typeof(PropertyType));
+            typeCfg.KeepDeserialized = true;
+
+            PortableConfiguration cfg = new PortableConfiguration();
+            cfg.DefaultKeepDeserialized = false;
+
+            cfg.TypeConfigurations = new List<PortableTypeConfiguration> { typeCfg };
+
+            CheckKeepSerialized(cfg, true);
+        }
+
+        /// <summary>
+        /// Test correct serialization/deserialization of arrays of special types.
+        /// </summary>
+        [Test]
+        public void TestSpecialArrays()
+        {
+            ICollection<PortableTypeConfiguration> typeCfgs =
+                new List<PortableTypeConfiguration>();
+
+            typeCfgs.Add(new PortableTypeConfiguration(typeof(SpecialArray)));
+            typeCfgs.Add(new PortableTypeConfiguration(typeof(SpecialArrayMarshalAware)));
+
+            PortableConfiguration cfg = new PortableConfiguration();
+
+            cfg.TypeConfigurations = typeCfgs;
+
+            PortableMarshaller marsh = new PortableMarshaller(cfg);
+
+            Guid[] guidArr = { Guid.NewGuid() };
+            Guid?[] nGuidArr = { Guid.NewGuid() };
+            DateTime[] dateArr = { DateTime.Now.ToUniversalTime() };
+            DateTime?[] nDateArr = { DateTime.Now.ToUniversalTime() };
+
+            // Use special object.
+            SpecialArray obj1 = new SpecialArray();
+
+            obj1.GuidArr = guidArr;
+            obj1.NGuidArr = nGuidArr;
+            obj1.DateArr = dateArr;
+            obj1.NDateArr = nDateArr;
+
+            byte[] bytes = marsh.Marshal(obj1);
+
+            IPortableObject portObj = marsh.Unmarshal<IPortableObject>(bytes, PortableMode.ForcePortable);
+
+            Assert.AreEqual(guidArr, portObj.Field<Guid[]>("guidArr"));
+            Assert.AreEqual(nGuidArr, portObj.Field<Guid?[]>("nGuidArr"));
+            Assert.AreEqual(dateArr, portObj.Field<DateTime[]>("dateArr"));
+            Assert.AreEqual(nDateArr, portObj.Field<DateTime?[]>("nDateArr"));
+
+            obj1 = portObj.Deserialize<SpecialArray>();
+
+            Assert.AreEqual(guidArr, obj1.GuidArr);
+            Assert.AreEqual(nGuidArr, obj1.NGuidArr);
+            Assert.AreEqual(dateArr, obj1.DateArr);
+            Assert.AreEqual(nDateArr, obj1.NDateArr);
+
+            // Use special with IGridPortableMarshalAware.
+            SpecialArrayMarshalAware obj2 = new SpecialArrayMarshalAware();
+
+            obj2.GuidArr = guidArr;
+            obj2.NGuidArr = nGuidArr;
+            obj2.DateArr = dateArr;
+            obj2.NDateArr = nDateArr;
+
+            bytes = marsh.Marshal(obj2);
+
+            portObj = marsh.Unmarshal<IPortableObject>(bytes, PortableMode.ForcePortable);
+
+            Assert.AreEqual(guidArr, portObj.Field<Guid[]>("a"));
+            Assert.AreEqual(nGuidArr, portObj.Field<Guid?[]>("b"));
+            Assert.AreEqual(dateArr, portObj.Field<DateTime[]>("c"));
+            Assert.AreEqual(nDateArr, portObj.Field<DateTime?[]>("d"));
+
+            obj2 = portObj.Deserialize<SpecialArrayMarshalAware>();
+
+            Assert.AreEqual(guidArr, obj2.GuidArr);
+            Assert.AreEqual(nGuidArr, obj2.NGuidArr);
+            Assert.AreEqual(dateArr, obj2.DateArr);
+            Assert.AreEqual(nDateArr, obj2.NDateArr);
+        }
+
+        private static void CheckKeepSerialized(PortableConfiguration cfg, bool expKeep)
+        {
+            if (cfg.TypeConfigurations == null)
+            {
+                cfg.TypeConfigurations = new List<PortableTypeConfiguration>
+                {
+                    new PortableTypeConfiguration(typeof(PropertyType))
+                };
+            }
+
+            PortableMarshaller marsh = new PortableMarshaller(cfg);
+
+            byte[] data = marsh.Marshal(new PropertyType());
+
+            IPortableObject portNewObj = marsh.Unmarshal<IPortableObject>(data, PortableMode.ForcePortable);
+
+            PropertyType deserialized1 = portNewObj.Deserialize<PropertyType>();
+            PropertyType deserialized2 = portNewObj.Deserialize<PropertyType>();
+
+            Assert.NotNull(deserialized1);
+
+            Assert.AreEqual(expKeep, deserialized1 == deserialized2);
+        }
+
+        private void CheckHandlesConsistency(HandleOuter outer, HandleInner inner, HandleOuter newOuter, 
+            HandleInner newInner)
+        {
+            Assert.True(newOuter != null);
+            Assert.AreEqual(outer.Before, newOuter.Before);
+            Assert.True(newOuter.Inner == newInner);
+            Assert.AreEqual(outer.After, newOuter.After);
+            Assert.AreEqual(outer.RawBefore, newOuter.RawBefore);
+            Assert.True(newOuter.RawInner == newInner);
+            Assert.AreEqual(outer.RawAfter, newOuter.RawAfter);
+
+            Assert.True(newInner != null);
+            Assert.AreEqual(inner.Before, newInner.Before);
+            Assert.True(newInner.Outer == newOuter);
+            Assert.AreEqual(inner.After, newInner.After);
+            Assert.AreEqual(inner.RawBefore, newInner.RawBefore);
+            Assert.True(newInner.RawOuter == newOuter);
+            Assert.AreEqual(inner.RawAfter, newInner.RawAfter);            
+        }
+
+        private static void CheckObject(PortableMarshaller marsh, OuterObjectType outObj, InnerObjectType inObj)
+        {
+            inObj.PInt1 = 1;
+            inObj.PInt2 = 2;
+
+            outObj.InObj = inObj;
+
+            byte[] bytes = marsh.Marshal(outObj);
+
+            IPortableObject portOutObj = marsh.Unmarshal<IPortableObject>(bytes, PortableMode.ForcePortable);
+
+            Assert.AreEqual(outObj.GetHashCode(), portOutObj.GetHashCode());
+
+            OuterObjectType newOutObj = portOutObj.Deserialize<OuterObjectType>();
+
+            Assert.AreEqual(outObj, newOutObj);
+        }
+
+        public class OuterObjectType
+        {
+            public InnerObjectType InObj { get; set; }
+
+            /** <inheritdoc /> */
+            public override bool Equals(object obj)
+            {
+                if (this == obj)
+                    return true;
+
+                var type = obj as OuterObjectType;
+                
+                return type != null && Equals(InObj, type.InObj);
+            }
+
+            /** <inheritdoc /> */
+            public override int GetHashCode()
+            {
+                return InObj != null ? InObj.GetHashCode() : 0;
+            }
+        }
+
+        public class InnerObjectType
+        {
+            public int PInt1 { get; set; }
+
+            public int PInt2 { get; set; }
+
+            /** <inheritdoc /> */
+            public override bool Equals(object obj)
+            {
+                if (this == obj)
+                    return true;
+
+                var that = obj as InnerObjectType;
+
+                return that != null && (PInt1 == that.PInt1 && PInt2 == that.PInt2);
+            }
+
+            /** <inheritdoc /> */
+            public override int GetHashCode()
+            {
+                return 31 * PInt1 + PInt2;
+            }
+
+            /** <inheritdoc /> */
+            public override string ToString()
+            {
+                return "InnerObjectType[pInt1=" + PInt1 + ", pInt2=" + PInt2 + ']';
+            }
+        }
+
+        public class CollectionsType
+        {
+            public ICollection Col1 { get; set; }
+
+            public ArrayList Col2 { get; set; }
+
+            /** <inheritdoc /> */
+            public override bool Equals(object obj)
+            {
+                if (this == obj)
+                    return true;
+
+                if (obj != null && obj is CollectionsType)
+                {
+                    CollectionsType that = (CollectionsType)obj;
+
+                    return CompareCollections(Col1, that.Col1) && CompareCollections(Col2, that.Col2);
+                }
+                return false;
+            }
+
+            /** <inheritdoc /> */
+            public override int GetHashCode()
+            {
+                int res = Col1 != null ? Col1.GetHashCode() : 0;
+
+                res = 31 * res + (Col2 != null ? Col2.GetHashCode() : 0);
+
+                return res;
+            }
+
+            /** <inheritdoc /> */
+            public override string ToString()
+            {
+                return "CollectoinsType[col1=" + CollectionAsString(Col1) + 
+                    ", col2=" + CollectionAsString(Col2) + ']'; 
+            }
+        }
+
+        private static string CollectionAsString(ICollection col)
+        {
+            if (col == null)
+                return null;
+            StringBuilder sb = new StringBuilder("[");
+
+            bool first = true;
+
+            foreach (object elem in col)
+            {
+                if (first)
+                    first = false;
+                else
+                    sb.Append(", ");
+
+                sb.Append(elem);
+            }
+
+            sb.Append("]");
+
+            return sb.ToString();
+        }
+
+        public class TestList : ArrayList
+        {
+
+        }
+
+        private static bool CompareCollections(ICollection col1, ICollection col2)
+        {
+            if (col1 == null && col2 == null)
+                return true;
+            if (col1 == null || col2 == null)
+                return false;
+
+            return col1.OfType<object>().SequenceEqual(col2.OfType<object>());
+        }
+
+        public class PrimitiveArrayFieldType
+        {
+            public bool[] PBool { get; set; }
+
+            public sbyte[] PSbyte { get; set; }
+
+            public byte[] PByte { get; set; }
+
+            public short[] PShort { get; set; }
+
+            public ushort[] PUshort { get; set; }
+
+            public char[] PChar { get; set; }
+
+            public int[] PInt { get; set; }
+
+            public uint[] PUint { get; set; }
+
+            public long[] PLong { get; set; }
+
+            public ulong[] PUlong { get; set; }
+
+            public float[] PFloat { get; set; }
+
+            public double[] PDouble { get; set; }
+
+            public string[] PString { get; set; }
+
+            public Guid?[] PGuid { get; set; }
+
+            /** <inheritdoc /> */
+            public override bool Equals(object obj)
+            {
+                if (this == obj)
+                    return true;
+
+                var other = obj as PrimitiveArrayFieldType;
+
+                return other != null && (PBool == other.PBool &&
+                                         PByte == other.PByte &&
+                                         PSbyte == other.PSbyte &&
+                                         PShort == other.PShort &&
+                                         PUshort == other.PUshort &&
+                                         PInt == other.PInt &&
+                                         PUint == other.PUint &&
+                                         PLong == other.PLong &&
+                                         PUlong == other.PUlong &&
+                                         PChar == other.PChar &&
+                                         PFloat == other.PFloat &&
+                                         PDouble == other.PDouble &&
+                                         PString == other.PString &&
+                                         PGuid == other.PGuid);
+            }
+
+            /** <inheritdoc /> */
+            public override int GetHashCode()
+            {
+                return PInt != null && PInt.Length > 0 ? PInt[0].GetHashCode() : 0;
+            }
+        }
+
+        public class SpecialArray
+        {
+            public Guid[] GuidArr;
+            public Guid?[] NGuidArr;
+            public DateTime[] DateArr;
+            public DateTime?[] NDateArr;
+        }
+
+        public class SpecialArrayMarshalAware : SpecialArray, IPortableMarshalAware
+        {
+            public void WritePortable(IPortableWriter writer)
+            {
+                writer.WriteObjectArray("a", GuidArr);
+                writer.WriteObjectArray("b", NGuidArr);
+                writer.WriteObjectArray("c", DateArr);
+                writer.WriteObjectArray("d", NDateArr);
+            }
+
+            public void ReadPortable(IPortableReader reader)
+            {
+                GuidArr = reader.ReadObjectArray<Guid>("a");
+                NGuidArr = reader.ReadObjectArray<Guid?>("b");
+                DateArr = reader.ReadObjectArray<DateTime>("c");
+                NDateArr = reader.ReadObjectArray<DateTime?>("d");
+            }
+        }
+
+        public class EnumType
+        {
+            public TestEnum PEnum { get; set; }
+
+            public TestEnum[] PEnumArray { get; set; }
+        }
+
+        public class PrimitiveFieldType 
+        {
+            private Guid _pGuid;
+
+            public bool PBool { get; set; }
+
+            public sbyte PSbyte { get; set; }
+
+            public byte PByte { get; set; }
+
+            public short PShort { get; set; }
+
+            public ushort PUshort { get; set; }
+
+            public char PChar { get; set; }
+
+            public int PInt { get; set; }
+
+            public uint PUint { get; set; }
+
+            public long PLong { get; set; }
+
+            public ulong PUlong { get; set; }
+
+            public float PFloat { get; set; }
+
+            public double PDouble { get; set; }
+
+            public string PString { get; set; }
+
+            public Guid PGuid
+            {
+                get { return _pGuid; }
+                set { _pGuid = value; }
+            }
+
+            public Guid? PnGuid { get; set; }
+
+            /** <inheritdoc /> */
+            public override bool Equals(object obj)
+            {
+                if (this == obj)
+                    return true;
+
+                if (obj != null && obj is PrimitiveFieldType)
+                {
+                    PrimitiveFieldType that = (PrimitiveFieldType)obj;
+
+                    return PBool == that.PBool &&
+                        PByte == that.PByte &&
+                        PSbyte == that.PSbyte &&
+                        PShort == that.PShort &&
+                        PUshort == that.PUshort &&
+                        PInt == that.PInt &&
+                        PUint == that.PUint &&
+                        PLong == that.PLong &&
+                        PUlong == that.PUlong &&
+                        PChar == that.PChar &&
+                        PFloat == that.PFloat &&
+                        PDouble == that.PDouble &&
+                        (PString == null && that.PString == null || PString != null && PString.Equals(that.PString)) &&
+                        _pGuid.Equals(that._pGuid) &&
+                        (PnGuid == null && that.PnGuid == null || PnGuid != null && PnGuid.Equals(that.PnGuid));
+                }
+                return false;
+            }
+
+            /** <inheritdoc /> */
+            public override int GetHashCode()
+            {
+                return PInt;
+            }
+        }
+
+        public class PrimitiveFieldPortableType : PrimitiveFieldType, IPortableMarshalAware
+        {
+            public unsafe void WritePortable(IPortableWriter writer)
+            {
+                writer.WriteBoolean("bool", PBool);
+                writer.WriteByte("byte", PByte);
+                writer.WriteShort("short", PShort);
+                writer.WriteInt("int", PInt);
+                writer.WriteLong("long", PLong);
+                writer.WriteChar("char", PChar);
+                writer.WriteFloat("float", PFloat);
+                writer.WriteDouble("double", PDouble);
+
+                sbyte sByte = PSbyte;
+                ushort uShort = PUshort;
+                uint uInt = PUint;
+                ulong uLong = PUlong;
+
+                writer.WriteByte("sbyte", *(byte*)&sByte);
+                writer.WriteShort("ushort", *(short*)&uShort);
+                writer.WriteInt("uint", *(int*)&uInt);
+                writer.WriteLong("ulong", *(long*)&uLong);
+
+                writer.WriteString("string", PString);
+                writer.WriteGuid("guid", PGuid);
+                writer.WriteGuid("nguid", PnGuid);
+            }
+
+            public unsafe void ReadPortable(IPortableReader reader)
+            {
+                PBool = reader.ReadBoolean("bool");
+                PByte = reader.ReadByte("byte");
+                PShort = reader.ReadShort("short");
+                PInt = reader.ReadInt("int");
+
+                PLong = reader.ReadLong("long");
+                PChar = reader.ReadChar("char");
+                PFloat = reader.ReadFloat("float");
+                PDouble = reader.ReadDouble("double");
+
+                byte sByte = reader.ReadByte("sbyte");
+                short uShort = reader.ReadShort("ushort");
+                int uInt = reader.ReadInt("uint");
+                long uLong = reader.ReadLong("ulong");
+
+                PSbyte = *(sbyte*)&sByte;
+                PUshort = *(ushort*)&uShort;
+                PUint = *(uint*)&uInt;
+                PUlong = *(ulong*)&uLong;
+
+                PString = reader.ReadString("string");
+                PGuid = reader.ReadGuid("guid").Value;
+                PnGuid = reader.ReadGuid("nguid");
+            }
+        }
+
+        public class PrimitiveFieldRawPortableType : PrimitiveFieldType, IPortableMarshalAware
+        {
+            public unsafe void WritePortable(IPortableWriter writer)
+            {
+                IPortableRawWriter rawWriter = writer.RawWriter();
+
+                rawWriter.WriteBoolean(PBool);
+                rawWriter.WriteByte(PByte);
+                rawWriter.WriteShort(PShort);
+                rawWriter.WriteInt(PInt);
+                rawWriter.WriteLong(PLong);
+                rawWriter.WriteChar(PChar);
+                rawWriter.WriteFloat(PFloat);
+                rawWriter.WriteDouble(PDouble);
+
+                sbyte sByte = PSbyte;
+                ushort uShort = PUshort;
+                uint uInt = PUint;
+                ulong uLong = PUlong;
+
+                rawWriter.WriteByte(*(byte*)&sByte);
+                rawWriter.WriteShort(*(short*)&uShort);
+                rawWriter.WriteInt(*(int*)&uInt);
+                rawWriter.WriteLong(*(long*)&uLong);
+
+                rawWriter.WriteString(PString);
+                rawWriter.WriteGuid(PGuid);
+                rawWriter.WriteGuid(PnGuid);
+            }
+
+            public unsafe void ReadPortable(IPortableReader reader)
+            {
+                IPortableRawReader rawReader = reader.RawReader();
+
+                PBool = rawReader.ReadBoolean();
+                PByte = rawReader.ReadByte();
+                PShort = rawReader.ReadShort();
+                PInt = rawReader.ReadInt();
+
+                PLong = rawReader.ReadLong();
+                PChar = rawReader.ReadChar();
+                PFloat = rawReader.ReadFloat();
+                PDouble = rawReader.ReadDouble();
+
+                byte sByte = rawReader.ReadByte();
+                short uShort = rawReader.ReadShort();
+                int uInt = rawReader.ReadInt();
+                long uLong = rawReader.ReadLong();
+
+                PSbyte = *(sbyte*)&sByte;
+                PUshort = *(ushort*)&uShort;
+                PUint = *(uint*)&uInt;
+                PUlong = *(ulong*)&uLong;
+
+                PString = rawReader.ReadString();
+                PGuid = rawReader.ReadGuid().Value;
+                PnGuid = rawReader.ReadGuid();
+            }
+        }
+
+        public class PrimitiveFieldsSerializer : IPortableSerializer
+        {
+            public unsafe void WritePortable(object obj, IPortableWriter writer)
+            {
+                PrimitiveFieldType obj0 = (PrimitiveFieldType)obj;
+
+                writer.WriteBoolean("bool", obj0.PBool);
+                writer.WriteByte("byte", obj0.PByte);
+                writer.WriteShort("short", obj0.PShort);
+                writer.WriteInt("int", obj0.PInt);
+                writer.WriteLong("long", obj0.PLong);
+                writer.WriteChar("char", obj0.PChar);
+                writer.WriteFloat("float", obj0.PFloat);
+                writer.WriteDouble("double", obj0.PDouble);
+
+                sbyte sByte = obj0.PSbyte;
+                ushort uShort = obj0.PUshort;
+                uint uInt = obj0.PUint;
+                ulong uLong = obj0.PUlong;
+
+                writer.WriteByte("sbyte", *(byte*)&sByte);
+                writer.WriteShort("ushort", *(short*)&uShort);
+                writer.WriteInt("uint", *(int*)&uInt);
+                writer.WriteLong("ulong", *(long*)&uLong);
+
+                writer.WriteString("string", obj0.PString);
+                writer.WriteGuid("guid", obj0.PGuid);
+                writer.WriteGuid("nguid", obj0.PnGuid);
+            }
+
+            public unsafe void ReadPortable(object obj, IPortableReader reader)
+            {
+                PrimitiveFieldType obj0 = (PrimitiveFieldType)obj;
+
+                obj0.PBool = reader.ReadBoolean("bool");
+                obj0.PByte = reader.ReadByte("byte");
+                obj0.PShort = reader.ReadShort("short");
+                obj0.PInt = reader.ReadInt("int");
+
+                obj0.PLong = reader.ReadLong("long");
+                obj0.PChar = reader.ReadChar("char");
+                obj0.PFloat = reader.ReadFloat("float");
+                obj0.PDouble = reader.ReadDouble("double");
+
+                byte sByte = reader.ReadByte("sbyte");
+                short uShort = reader.ReadShort("ushort");
+                int uInt = reader.ReadInt("uint");
+                long uLong = reader.ReadLong("ulong");
+
+                obj0.PSbyte = *(sbyte*)&sByte;
+                obj0.PUshort = *(ushort*)&uShort;
+                obj0.PUint = *(uint*)&uInt;
+                obj0.PUlong = *(ulong*)&uLong;
+
+                obj0.PString = reader.ReadString("string");
+                obj0.PGuid = reader.ReadGuid("guid").Value;
+                obj0.PnGuid = reader.ReadGuid("nguid");
+            }
+        }
+
+        public class PrimitiveFieldsRawSerializer : IPortableSerializer
+        {
+            public unsafe void WritePortable(object obj, IPortableWriter writer)
+            {
+                PrimitiveFieldType obj0 = (PrimitiveFieldType)obj;
+
+                IPortableRawWriter rawWriter = writer.RawWriter();
+
+                rawWriter.WriteBoolean(obj0.PBool);
+                rawWriter.WriteByte(obj0.PByte);
+                rawWriter.WriteShort( obj0.PShort);
+                rawWriter.WriteInt( obj0.PInt);
+                rawWriter.WriteLong( obj0.PLong);
+                rawWriter.WriteChar(obj0.PChar);
+                rawWriter.WriteFloat(obj0.PFloat);
+                rawWriter.WriteDouble( obj0.PDouble);
+
+                sbyte sByte = obj0.PSbyte;
+                ushort uShort = obj0.PUshort;
+                uint uInt = obj0.PUint;
+                ulong uLong = obj0.PUlong;
+
+                rawWriter.WriteByte(*(byte*)&sByte);
+                rawWriter.WriteShort(*(short*)&uShort);
+                rawWriter.WriteInt(*(int*)&uInt);
+                rawWriter.WriteLong(*(long*)&uLong);
+
+                rawWriter.WriteString(obj0.PString);
+                rawWriter.WriteGuid(obj0.PGuid);
+                rawWriter.WriteGuid(obj0.PnGuid);
+            }
+
+            public unsafe void ReadPortable(object obj, IPortableReader reader)
+            {
+                PrimitiveFieldType obj0 = (PrimitiveFieldType)obj;
+
+                IPortableRawReader rawReader = reader.RawReader();
+
+                obj0.PBool = rawReader.ReadBoolean();
+                obj0.PByte = rawReader.ReadByte();
+                obj0.PShort = rawReader.ReadShort();
+                obj0.PInt = rawReader.ReadInt();
+                obj0.PLong = rawReader.ReadLong();
+                obj0.PChar = rawReader.ReadChar();
+                obj0.PFloat = rawReader.ReadFloat();
+                obj0.PDouble = rawReader.ReadDouble();
+
+                byte sByte = rawReader.ReadByte();
+                short uShort = rawReader.ReadShort();
+                int uInt = rawReader.ReadInt();
+                long uLong = rawReader.ReadLong();
+
+                obj0.PSbyte = *(sbyte*)&sByte;
+                obj0.PUshort = *(ushort*)&uShort;
+                obj0.PUint = *(uint*)&uInt;
+                obj0.PUlong = *(ulong*)&uLong;
+
+                obj0.PString = rawReader.ReadString();
+                obj0.PGuid = rawReader.ReadGuid().Value;
+                obj0.PnGuid = rawReader.ReadGuid();
+            }
+        }
+
+        public static string PrintBytes(byte[] bytes)
+        {
+            StringBuilder sb = new StringBuilder();
+
+            foreach (byte b in bytes)
+                sb.Append(b + " ");
+
+            return sb.ToString();
+        }
+
+        public class HandleOuter : IPortableMarshalAware
+        {
+            public string Before;
+            public HandleInner Inner;
+            public string After;
+
+            public string RawBefore;
+            public HandleInner RawInner;
+            public string RawAfter;
+
+            /** <inheritdoc /> */
+            virtual public void WritePortable(IPortableWriter writer)
+            {
+                writer.WriteString("before", Before);
+                writer.WriteObject("inner", Inner);
+                writer.WriteString("after", After);
+
+                IPortableRawWriter rawWriter = writer.RawWriter();
+
+                rawWriter.WriteString(RawBefore);
+                rawWriter.WriteObject(RawInner);
+                rawWriter.WriteString(RawAfter);
+            }
+
+            /** <inheritdoc /> */
+            virtual public void ReadPortable(IPortableReader reader)
+            {
+                Before = reader.ReadString("before");
+                Inner = reader.ReadObject<HandleInner>("inner");
+                After = reader.ReadString("after");
+
+                IPortableRawReader rawReader = reader.RawReader();
+
+                RawBefore = rawReader.ReadString();
+                RawInner = rawReader.ReadObject<HandleInner>();
+                RawAfter = rawReader.ReadString();
+            }
+        }
+
+        public class HandleInner : IPortableMarshalAware
+        {
+            public string Before;
+            public HandleOuter Outer;
+            public string After;
+
+            public string RawBefore;
+            public HandleOuter RawOuter;
+            public string RawAfter;
+
+            /** <inheritdoc /> */
+            virtual public void WritePortable(IPortableWriter writer)
+            {
+                writer.WriteString("before", Before);
+                writer.WriteObject("outer", Outer);
+                writer.WriteString("after", After);
+
+                IPortableRawWriter rawWriter = writer.RawWriter();
+
+                rawWriter.WriteString(RawBefore);
+                rawWriter.WriteObject(RawOuter);
+                rawWriter.WriteString(RawAfter);
+            }
+
+            /** <inheritdoc /> */
+            virtual public void ReadPortable(IPortableReader reader)
+            {
+                Before = reader.ReadString("before");
+                Outer = reader.ReadObject<HandleOuter>("outer");
+                After = reader.ReadString("after");
+
+                IPortableRawReader rawReader = reader.RawReader();
+
+                RawBefore = rawReader.ReadString();
+                RawOuter = rawReader.ReadObject<HandleOuter>();
+                RawAfter = rawReader.ReadString();
+            }
+        }
+
+
+        public class HandleOuterExclusive : HandleOuter
+        {
+            /** <inheritdoc /> */
+            override public void WritePortable(IPortableWriter writer)
+            {
+                PortableWriterImpl writer0 = (PortableWriterImpl)writer;
+
+                writer.WriteString("before", Before);
+
+                writer0.DetachNext();
+                writer.WriteObject("inner", Inner);
+
+                writer.WriteString("after", After);
+
+                IPortableRawWriter rawWriter = writer.RawWriter();
+
+                rawWriter.WriteString(RawBefore);
+
+                writer0.DetachNext();
+                rawWriter.WriteObject(RawInner);
+
+                rawWriter.WriteString(RawAfter);
+            }
+
+            /** <inheritdoc /> */
+            override public void ReadPortable(IPortableReader reader)
+            {
+                var reader0 = (PortableReaderImpl) reader;
+
+                Before = reader0.ReadString("before");
+
+                reader0.DetachNext();
+                Inner = reader0.ReadObject<HandleInner>("inner");
+
+                After = reader0.ReadString("after");
+
+                var rawReader = (PortableReaderImpl) reader.RawReader();
+
+                RawBefore = rawReader.ReadString();
+
+                reader0.DetachNext();
+                RawInner = rawReader.ReadObject<HandleInner>();
+
+                RawAfter = rawReader.ReadString();
+            }
+        }
+
+        public class PropertyType
+        {
+            public int Field1;
+
+            public int Field2
+            {
+                get;
+                set;
+            }
+        }
+
+        public enum TestEnum
+        {
+            Val1, Val2, Val3 = 10
+        }
+
+        public class DecimalReflective
+        {
+            /** */
+            public decimal Val;
+
+            /** */
+            public decimal[] ValArr;
+        }
+
+        public class DecimalMarshalAware : DecimalReflective, IPortableMarshalAware
+        {
+            /** */
+            public decimal RawVal;
+
+            /** */
+            public decimal[] RawValArr;
+
+            /** <inheritDoc /> */
+            public void WritePortable(IPortableWriter writer)
+            {
+                writer.WriteDecimal("val", Val);
+                writer.WriteDecimalArray("valArr", ValArr);
+
+                IPortableRawWriter rawWriter = writer.RawWriter();
+
+                rawWriter.WriteDecimal(RawVal);
+                rawWriter.WriteDecimalArray(RawValArr);
+            }
+
+            /** <inheritDoc /> */
+            public void ReadPortable(IPortableReader reader)
+            {
+                Val = reader.ReadDecimal("val");
+                ValArr = reader.ReadDecimalArray("valArr");
+
+                IPortableRawReader rawReader = reader.RawReader();
+
+                RawVal = rawReader.ReadDecimal();
+                RawValArr = rawReader.ReadDecimalArray();
+            }
+        }
+
+        /// <summary>
+        /// Date time type.
+        /// </summary>
+        public class DateTimeType : IPortableMarshalAware
+        {
+            public DateTime Loc;
+            public DateTime Utc;
+
+            public DateTime? LocNull;
+            public DateTime? UtcNull;
+
+            public DateTime?[] LocArr;
+            public DateTime?[] UtcArr;
+
+            public DateTime LocRaw;
+            public DateTime UtcRaw;
+
+            public DateTime? LocNullRaw;
+            public DateTime? UtcNullRaw;
+
+            public DateTime?[] LocArrRaw;
+            public DateTime?[] UtcArrRaw;
+
+            /// <summary>
+            /// Constructor.
+            /// </summary>
+            /// <param name="now">Current local time.</param>
+            public DateTimeType(DateTime now)
+            {
+                Loc = now;
+                Utc = now.ToUniversalTime();
+
+                LocNull = Loc;
+                UtcNull = Utc;
+
+                LocArr = new DateTime?[] { Loc };
+                UtcArr = new DateTime?[] { Utc };
+
+                LocRaw = Loc;
+                UtcRaw = Utc;
+
+                LocNullRaw = LocNull;
+                UtcNullRaw = UtcNull;
+
+                LocArrRaw = new[] { LocArr[0] };
+                UtcArrRaw = new[] { UtcArr[0] };
+            }
+
+            /** <inheritDoc /> */
+            public void WritePortable(IPortableWriter writer)
+            {
+                writer.WriteDate("loc", Loc);
+                writer.WriteDate("utc", Utc);
+                writer.WriteDate("locNull", LocNull);
+                writer.WriteDate("utcNull", UtcNull);
+                writer.WriteDateArray("locArr", LocArr);
+                writer.WriteDateArray("utcArr", UtcArr);
+
+                IPortableRawWriter rawWriter = writer.RawWriter();
+
+                rawWriter.WriteDate(LocRaw);
+                rawWriter.WriteDate(UtcRaw);
+                rawWriter.WriteDate(LocNullRaw);
+                rawWriter.WriteDate(UtcNullRaw);
+                rawWriter.WriteDateArray(LocArrRaw);
+                rawWriter.WriteDateArray(UtcArrRaw);
+            }
+
+            /** <inheritDoc /> */
+            public void ReadPortable(IPortableReader reader)
+            {
+                Loc = reader.ReadDate("loc", true).Value;
+                Utc = reader.ReadDate("utc", false).Value;
+                LocNull = reader.ReadDate("loc", true).Value;
+                UtcNull = reader.ReadDate("utc", false).Value;
+                LocArr = reader.ReadDateArray("locArr", true);
+                UtcArr = reader.ReadDateArray("utcArr", false);
+
+                IPortableRawReader rawReader = reader.RawReader();
+
+                LocRaw = rawReader.ReadDate(true).Value;
+                UtcRaw = rawReader.ReadDate(false).Value;
+                LocNullRaw = rawReader.ReadDate(true).Value;
+                UtcNullRaw = rawReader.ReadDate(false).Value;
+                LocArrRaw = rawReader.ReadDateArray(true);
+                UtcArrRaw = rawReader.ReadDateArray(false);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/PortableConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/PortableConfigurationTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/PortableConfigurationTest.cs
new file mode 100644
index 0000000..67a8d7d
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/PortableConfigurationTest.cs
@@ -0,0 +1,173 @@
+/*
+ * 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.Collections.Generic;
+    using System.Linq;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Portable;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Portable configuration tests.
+    /// </summary>
+    public class PortableConfigurationTest
+    {
+        /** Cache. */
+        private ICache<int, TestGenericPortableBase> _cache;
+
+        /** Random generator. */
+        private static readonly Random Rnd = new Random();
+
+        /** Test types for code config */
+        private static readonly Type[] TestTypes = {
+            typeof (TestGenericPortable<int>),
+            typeof (TestGenericPortable<string>),
+            typeof (TestGenericPortable<TestGenericPortable<int>>),
+            typeof (TestGenericPortable<List<Tuple<int, string>>>),
+            typeof (TestGenericPortable<int, string>),
+            typeof (TestGenericPortable<int, TestGenericPortable<string>>),
+            typeof (TestGenericPortable<int, string, Type>),
+            typeof (TestGenericPortable<int, string, TestGenericPortable<int, string, Type>>)
+        };
+
+        /** Test types for xml config */
+        private static readonly Type[] TestTypesXml = {
+            typeof (TestGenericPortable<long>),
+            typeof (TestGenericPortable<Type>),
+            typeof (TestGenericPortable<TestGenericPortable<long>>),
+            typeof (TestGenericPortable<List<Tuple<long, string>>>),
+            typeof (TestGenericPortable<long, string>),
+            typeof (TestGenericPortable<long, TestGenericPortable<string>>),
+            typeof (TestGenericPortable<long, string, Type>),
+            typeof (TestGenericPortable<long, string, TestGenericPortable<long, string, Type>>)
+        };
+
+        /// <summary>
+        /// Starts the grid with provided config.
+        /// </summary>
+        /// <param name="portableConfiguration">The portable configuration.</param>
+        private void StartGrid(PortableConfiguration portableConfiguration)
+        {
+            Ignition.StopAll(true);
+
+            var grid = Ignition.Start(new IgniteConfiguration
+            {
+                SpringConfigUrl = "config\\cache-portables.xml",
+                JvmClasspath = TestUtils.CreateTestClasspath(),
+                JvmOptions = TestUtils.TestJavaOptions(),
+                PortableConfiguration = portableConfiguration
+            });
+
+            _cache = grid.Cache<int, TestGenericPortableBase>(null);
+        }
+
+        /// <summary>
+        /// Test fixture tear-down routine.
+        /// </summary>
+        [TestFixtureTearDown]
+        public void TestFixtureTearDown()
+        {
+            TestUtils.KillProcesses();
+        }
+
+        /// <summary>
+        /// Tests the configuration set in code.
+        /// </summary>
+        [Test]
+        public void TestCodeConfiguration()
+        {
+            StartGrid(new PortableConfiguration
+            {
+                TypeConfigurations = TestTypes.Select(x => new PortableTypeConfiguration(x)).ToList()
+            });
+
+            CheckPortableTypes(TestTypes);
+        }
+
+        /// <summary>
+        /// Tests the configuration set in xml.
+        /// </summary>
+        [Test]
+        public void TestXmlConfiguration()
+        {
+            StartGrid(null);
+
+            CheckPortableTypes(TestTypesXml);
+        }
+
+        /// <summary>
+        /// Checks that specified types are portable and can be successfully used in cache.
+        /// </summary>
+        private void CheckPortableTypes(IEnumerable<Type> testTypes)
+        {
+            int key = 0;
+
+            foreach (var typ in testTypes)
+            {
+                key += 1;
+
+                var inst = CreateInstance(typ);
+
+                _cache.Put(key, inst);
+
+                var result = _cache.Get(key);
+
+                Assert.AreEqual(inst.Prop, result.Prop);
+
+                Assert.AreEqual(typ, result.GetType());
+            }
+        }
+
+        /// <summary>
+        /// Creates the instance of specified test portable type and sets a value on it.
+        /// </summary>
+        private static TestGenericPortableBase CreateInstance(Type type)
+        {
+            var inst = (TestGenericPortableBase)Activator.CreateInstance(type);
+
+            inst.Prop = Rnd.Next(int.MaxValue);
+
+            return inst;
+        }
+    }
+
+    public abstract class TestGenericPortableBase
+    {
+        public object Prop { get; set; }
+    }
+
+    public class TestGenericPortable<T> : TestGenericPortableBase
+    {
+        public T Prop1 { get; set; }
+    }
+
+    public class TestGenericPortable<T1, T2> : TestGenericPortableBase
+    {
+        public T1 Prop1 { get; set; }
+        public T2 Prop2 { get; set; }
+    }
+
+    public class TestGenericPortable<T1, T2, T3> : TestGenericPortableBase
+    {
+        public T1 Prop1 { get; set; }
+        public T2 Prop2 { get; set; }
+        public T3 Prop3 { get; set; }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Process/IIgniteProcessOutputReader.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Process/IIgniteProcessOutputReader.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Process/IIgniteProcessOutputReader.cs
new file mode 100644
index 0000000..d910c78
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Process/IIgniteProcessOutputReader.cs
@@ -0,0 +1,35 @@
+/*
+ * 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.Process
+{
+    using System.Diagnostics;
+
+    /// <summary>
+    /// Process output reader.
+    /// </summary>
+    public interface IIgniteProcessOutputReader
+    {
+        /// <summary>
+        /// Callback invoked when output data appear.
+        /// </summary>
+        /// <param name="proc">Process produced data.</param>
+        /// <param name="data">Data.</param>
+        /// <param name="err">Error flag.</param>
+        void OnOutput(Process proc, string data, bool err);
+    }
+}


[03/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcess.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcess.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcess.cs
new file mode 100644
index 0000000..9e332fe
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcess.cs
@@ -0,0 +1,292 @@
+/*
+ * 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.Process
+{
+    using System;
+    using System.Diagnostics;
+    using System.IO;
+    using System.Linq;
+    using System.Text;
+    using System.Threading;
+    using Apache.Ignite.Core.Impl;
+
+    /// <summary>
+    /// Defines forked Ignite node.
+    /// </summary>
+    public class IgniteProcess
+    {
+        /** Executable file name. */
+        private static readonly string ExeName = "Ignite.exe";
+
+        /** Executable process name. */
+        private static readonly string ExeProcName = ExeName.Substring(0, ExeName.IndexOf('.'));
+
+        /** Executable configuration file name. */
+        private static readonly string ExeCfgName = ExeName + ".config";
+
+        /** Executable backup configuration file name. */
+        private static readonly string ExeCfgBakName = ExeCfgName + ".bak";
+
+        /** Directory where binaries are stored. */
+        private static readonly string ExeDir;
+
+        /** Full path to executable. */
+        private static readonly string ExePath;
+
+        /** Full path to executable configuration file. */
+        private static readonly string ExeCfgPath;
+
+        /** Full path to executable configuration file backup. */
+        private static readonly string ExeCfgBakPath;
+
+        /** Default process output reader. */
+        private static readonly IIgniteProcessOutputReader DfltOutReader = new IgniteProcessConsoleOutputReader();
+
+        /** Process. */
+        private readonly Process _proc;
+
+        /// <summary>
+        /// Static initializer.
+        /// </summary>
+        static IgniteProcess()
+        {
+            // 1. Locate executable file and related stuff.
+            DirectoryInfo dir = new FileInfo(new Uri(typeof(IgniteProcess).Assembly.CodeBase).LocalPath).Directory;
+
+            // ReSharper disable once PossibleNullReferenceException
+            ExeDir = dir.FullName;
+
+            FileInfo[] exe = dir.GetFiles(ExeName);
+
+
+            // TODO: IGNITE-1367
+            /*
+            if (exe.Length == 0)
+                throw new Exception(ExeName + " is not found in test output directory: " + dir.FullName);
+
+            ExePath = exe[0].FullName;
+
+            FileInfo[] exeCfg = dir.GetFiles(ExeCfgName);
+
+            if (exeCfg.Length == 0)
+                throw new Exception(ExeCfgName + " is not found in test output directory: " + dir.FullName);
+
+            ExeCfgPath = exeCfg[0].FullName;
+
+            ExeCfgBakPath = Path.Combine(ExeDir, ExeCfgBakName);
+
+            File.Delete(ExeCfgBakPath);*/
+        }
+
+        /// <summary>
+        /// Save current configuration to backup.
+        /// </summary>
+        public static void SaveConfigurationBackup()
+        {
+            File.Copy(ExeCfgPath, ExeCfgBakPath, true);
+        }
+
+        /// <summary>
+        /// Restore configuration from backup.
+        /// </summary>
+        public static void RestoreConfigurationBackup()
+        {
+            File.Copy(ExeCfgBakPath, ExeCfgPath, true);
+        }
+
+        /// <summary>
+        /// Replace application configuration with another one.
+        /// </summary>
+        /// <param name="relPath">Path to config relative to executable directory.</param>
+        public static void ReplaceConfiguration(string relPath)
+        {
+            File.Copy(Path.Combine(ExeDir, relPath), ExeCfgPath, true);
+        }
+
+        /// <summary>
+        /// Kill all GridGain processes.
+        /// </summary>
+        public static void KillAll()
+        {
+            foreach (Process proc in Process.GetProcesses())
+            {
+                if (proc.ProcessName.Equals(ExeProcName))
+                {
+                    proc.Kill();
+
+                    proc.WaitForExit();
+                }
+            }
+        }
+
+        /// <summary>
+        /// Construector.
+        /// </summary>
+        /// <param name="args">Arguments</param>
+        public IgniteProcess(params string[] args) : this(DfltOutReader, args) { }
+
+        /// <summary>
+        /// Construector.
+        /// </summary>
+        /// <param name="outReader">Output reader.</param>
+        /// <param name="args">Arguments.</param>
+        public IgniteProcess(IIgniteProcessOutputReader outReader, params string[] args)
+        {
+            // Add test dll path
+            args = args.Concat(new[] {"-assembly=" + GetType().Assembly.Location}).ToArray();
+
+            _proc = Start(ExePath, IgniteManager.GetIgniteHome(null), outReader, args);
+        }
+
+        /// <summary>
+        /// Starts a grid process.
+        /// </summary>
+        /// <param name="exePath">Exe path.</param>
+        /// <param name="ggHome">GridGain home.</param>
+        /// <param name="outReader">Output reader.</param>
+        /// <param name="args">Arguments.</param>
+        /// <returns>Started process.</returns>
+        public static Process Start(string exePath, string ggHome, IIgniteProcessOutputReader outReader = null, 
+            params string[] args)
+        {
+            Debug.Assert(!string.IsNullOrEmpty(exePath));
+            Debug.Assert(!string.IsNullOrEmpty(ggHome));
+
+            // 1. Define process start configuration.
+            var sb = new StringBuilder();
+
+            foreach (string arg in args)
+                sb.Append('\"').Append(arg).Append("\" ");
+
+            var procStart = new ProcessStartInfo
+            {
+                FileName = exePath,
+                Arguments = sb.ToString()
+            };
+
+            if (!string.IsNullOrEmpty(ggHome))
+                procStart.EnvironmentVariables[IgniteManager.EnvIgniteHome] = ggHome;
+
+            procStart.EnvironmentVariables["GRIDGAIN_NATIVE_TEST_CLASSPATH"] = "true";
+
+            procStart.CreateNoWindow = true;
+            procStart.UseShellExecute = false;
+
+            procStart.RedirectStandardOutput = true;
+            procStart.RedirectStandardError = true;
+
+            var workDir = Path.GetDirectoryName(exePath);
+
+            if (workDir != null)
+                procStart.WorkingDirectory = workDir;
+
+            Console.WriteLine("About to run Ignite.exe process [exePath=" + exePath + ", arguments=" + sb + ']');
+
+            // 2. Start.
+            var proc = Process.Start(procStart);
+
+            Debug.Assert(proc != null);
+
+            // 3. Attach output readers to avoid hangs.
+            outReader = outReader ?? DfltOutReader;
+
+            Attach(proc, proc.StandardOutput, outReader, false);
+            Attach(proc, proc.StandardError, outReader, true);
+
+            return proc;
+        }
+
+        /// <summary>
+        /// Whether the process is still alive.
+        /// </summary>
+        public bool Alive
+        {
+            get
+            {
+                return !_proc.HasExited;
+            }
+        }
+
+        /// <summary>
+        /// Kill process.
+        /// </summary>
+        public void Kill()
+        {
+            _proc.Kill();
+        }
+
+        /// <summary>
+        /// Join process.
+        /// </summary>
+        /// <returns>Exit code.</returns>
+        public int Join()
+        {
+            _proc.WaitForExit();
+
+            return _proc.ExitCode;
+        }
+
+        /// <summary>
+        /// Join process with timeout.
+        /// </summary>
+        /// <param name="timeout">Timeout in milliseconds.</param>
+        /// <returns><c>True</c> if process exit occurred before timeout.</returns>
+        public bool Join(int timeout)
+        {
+            return _proc.WaitForExit(timeout);
+        }
+
+        /// <summary>
+        /// Join process with timeout.
+        /// </summary>
+        /// <param name="timeout">Timeout in milliseconds.</param>
+        /// <param name="exitCode">Exit code.</param>
+        /// <returns><c>True</c> if process exit occurred before timeout.</returns>
+        public bool Join(int timeout, out int exitCode)
+        {
+            if (_proc.WaitForExit(timeout))
+            {
+                exitCode = _proc.ExitCode;
+
+                return true;
+            }
+            exitCode = 0;
+
+            return false;
+        }
+
+        /// <summary>
+        /// Attach output reader to the process.
+        /// </summary>
+        /// <param name="proc">Process.</param>
+        /// <param name="reader">Process stream reader.</param>
+        /// <param name="outReader">Output reader.</param>
+        /// <param name="err">Whether this is error stream.</param>
+        private static void Attach(Process proc, StreamReader reader, IIgniteProcessOutputReader outReader, bool err)
+        {
+            Thread thread = new Thread(() =>
+            {
+                while (!proc.HasExited)
+                    outReader.OnOutput(proc, reader.ReadLine(), err);
+            }) {IsBackground = true};
+
+
+            thread.Start();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcessConsoleOutputReader.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcessConsoleOutputReader.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcessConsoleOutputReader.cs
new file mode 100644
index 0000000..00cc040
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcessConsoleOutputReader.cs
@@ -0,0 +1,40 @@
+/*
+ * 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.Process
+{
+    using System;
+    using System.Diagnostics;
+
+    /// <summary>
+    /// Output reader pushing data to the console.
+    /// </summary>
+    public class IgniteProcessConsoleOutputReader : IIgniteProcessOutputReader
+    {
+        /** Out message format. */
+        private static readonly string OutFormat = ">>> {0} OUT: {1}";
+
+        /** Error message format. */
+        private static readonly string ErrFormat = ">>> {0} ERR: {1}";
+
+        /** <inheritDoc /> */
+        public void OnOutput(Process proc, string data, bool err)
+        {
+            Console.WriteLine(err ? ErrFormat : OutFormat, proc.Id, data);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Query/ImplicitPortablePerson.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Query/ImplicitPortablePerson.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Query/ImplicitPortablePerson.cs
new file mode 100644
index 0000000..f80c4eb
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Query/ImplicitPortablePerson.cs
@@ -0,0 +1,46 @@
+/*
+ * 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.Query
+{
+    /// <summary>
+    /// Test person.
+    /// </summary>
+    internal class ImplicitPortablePerson
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ImplicitPortablePerson"/> class.
+        /// </summary>
+        /// <param name="name">The name.</param>
+        /// <param name="age">The age.</param>
+        public ImplicitPortablePerson(string name, int age)
+        {
+            Name = name;
+            Age = age;
+        }
+
+        /// <summary>
+        /// Gets or sets the name.
+        /// </summary>
+        public string Name { get; set; }
+
+        /// <summary>
+        /// Gets or sets the age.
+        /// </summary>
+        public int Age { get; set; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Query/NoDefPortablePerson.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Query/NoDefPortablePerson.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Query/NoDefPortablePerson.cs
new file mode 100644
index 0000000..16bd07d
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Query/NoDefPortablePerson.cs
@@ -0,0 +1,35 @@
+/*
+ * 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.Query
+{
+    /// <summary>
+    /// Test person.
+    /// </summary>
+    internal class NoDefPortablePerson
+    {
+        /// <summary>
+        /// Gets or sets the name.
+        /// </summary>
+        public string Name { get; set; }
+
+        /// <summary>
+        /// Gets or sets the age.
+        /// </summary>
+        public int Age { get; set; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Query/PortablePerson.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Query/PortablePerson.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Query/PortablePerson.cs
new file mode 100644
index 0000000..1e11001
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Query/PortablePerson.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.Query
+{
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Test person.
+    /// </summary>
+    internal class PortablePerson : IPortableMarshalAware
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="PortablePerson"/> class.
+        /// </summary>
+        /// <param name="name">The name.</param>
+        /// <param name="age">The age.</param>
+        public PortablePerson(string name, int age)
+        {
+            Name = name;
+            Age = age;
+        }
+
+        /// <summary>
+        /// Gets or sets the name.
+        /// </summary>
+        public string Name { get; set; }
+
+        /// <summary>
+        /// Gets or sets the address.
+        /// </summary>
+        public string Address { get; set; }
+
+        /// <summary>
+        /// Gets or sets the age.
+        /// </summary>
+        public int Age { get; set; }
+
+        /** <ineritdoc /> */
+        public void WritePortable(IPortableWriter writer)
+        {
+            writer.WriteString("name", Name);
+            writer.WriteString("address", Address);
+            writer.WriteInt("age", Age);
+        }
+
+        /** <ineritdoc /> */
+        public void ReadPortable(IPortableReader reader)
+        {
+            Name = reader.ReadString("name");
+            Address = reader.ReadString("address");
+            Age = reader.ReadInt("age");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/SerializationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/SerializationTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/SerializationTest.cs
new file mode 100644
index 0000000..8ed2899
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/SerializationTest.cs
@@ -0,0 +1,240 @@
+/*
+ * 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.Collections.Generic;
+    using System.Linq;
+    using System.Reflection;
+    using System.Reflection.Emit;
+    using System.Runtime.Serialization;
+    using System.Xml;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Impl;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests for native serialization.
+    /// </summary>
+    public class SerializationTest
+    {
+        /** Grid name. */
+        private const string GridName = "SerializationTest";
+
+        /// <summary>
+        /// Set up routine.
+        /// </summary>
+        [TestFixtureSetUp]
+        public void SetUp()
+        {
+            var cfg = new IgniteConfigurationEx
+            {
+                GridName = GridName,
+                JvmClasspath = TestUtils.CreateTestClasspath(),
+                JvmOptions = TestUtils.TestJavaOptions(),
+                SpringConfigUrl = "config\\native-client-test-cache.xml"
+            };
+
+            Ignition.Start(cfg);
+        }
+
+        /// <summary>
+        /// Tear down routine.
+        /// </summary>
+        [TestFixtureTearDown]
+        public void TearDown()
+        {
+            Ignition.StopAll(true);
+        }
+
+        /// <summary>
+        /// Test complex file serialization.
+        /// </summary>
+        [Test]
+        public void TestSerializableXmlDoc()
+        {
+            var grid = Ignition.GetIgnite(GridName);
+            var cache = grid.Cache<int, SerializableXmlDoc>("replicated");
+
+            var doc = new SerializableXmlDoc();
+
+            doc.LoadXml("<document><test1>val</test1><test2 attr=\"x\" /></document>");
+
+            for (var i = 0; i < 50; i++)
+            {
+                // Test cache
+                cache.Put(i, doc);
+
+                var resultDoc = cache.Get(i);
+
+                Assert.AreEqual(doc.OuterXml, resultDoc.OuterXml);
+
+                // Test task with document arg
+                CheckTask(grid, doc);
+            }
+        }
+
+        /// <summary>
+        /// Checks task execution.
+        /// </summary>
+        /// <param name="grid">Grid.</param>
+        /// <param name="arg">Task arg.</param>
+        private static void CheckTask(IIgnite grid, object arg)
+        {
+            var jobResult = grid.Compute().Execute(new CombineStringsTask(), arg);
+
+            var nodeCount = grid.Cluster.Nodes().Count;
+
+            var expectedRes =
+                CombineStringsTask.CombineStrings(Enumerable.Range(0, nodeCount).Select(x => arg.ToString()));
+
+            Assert.AreEqual(expectedRes, jobResult.InnerXml);
+        }
+
+        /// <summary>
+        /// Tests custom serialization binder.
+        /// </summary>
+        [Test]
+        public void TestSerializationBinder()
+        {
+            const int count = 50;
+
+            var cache = Ignition.GetIgnite(GridName).Cache<int, object>("local");
+
+            // Put multiple objects from muliple same-named assemblies to cache
+            for (var i = 0; i < count; i++)
+            {
+                dynamic val = Activator.CreateInstance(GenerateDynamicType());
+                
+                val.Id = i;
+                val.Name = "Name_" + i;
+
+                cache.Put(i, val);
+            }
+
+            // Verify correct deserialization
+            for (var i = 0; i < count; i++)
+            {
+                dynamic val = cache.Get(i);
+
+                Assert.AreEqual(val.Id, i);
+                Assert.AreEqual(val.Name, "Name_" + i);
+            }
+        }
+
+        /// <summary>
+        /// Generates a Type in runtime, puts it into a dynamic assembly.
+        /// </summary>
+        /// <returns></returns>
+        public static Type GenerateDynamicType()
+        {
+            var asmBuilder = AppDomain.CurrentDomain.DefineDynamicAssembly(
+                new AssemblyName("GridSerializationTestDynamicAssembly"), AssemblyBuilderAccess.Run);
+
+            var moduleBuilder = asmBuilder.DefineDynamicModule("GridSerializationTestDynamicModule");
+
+            var typeBuilder = moduleBuilder.DefineType("GridSerializationTestDynamicType",
+                TypeAttributes.Class | TypeAttributes.Public | TypeAttributes.Serializable);
+
+            typeBuilder.DefineField("Id", typeof (int), FieldAttributes.Public);
+            
+            typeBuilder.DefineField("Name", typeof (string), FieldAttributes.Public);
+
+            return typeBuilder.CreateType();
+        }
+    }
+
+    [Serializable]
+    [DataContract]
+    public sealed class SerializableXmlDoc : XmlDocument, ISerializable
+    {
+        /// <summary>
+        /// Default ctor.
+        /// </summary>
+        public SerializableXmlDoc()
+        {
+            // No-op
+        }
+
+        /// <summary>
+        /// Serialization ctor.
+        /// </summary>
+        private SerializableXmlDoc(SerializationInfo info, StreamingContext context)
+        {
+            LoadXml(info.GetString("xmlDocument"));
+        }
+
+        /** <inheritdoc /> */
+        public void GetObjectData(SerializationInfo info, StreamingContext context)
+        {
+            info.AddValue("xmlDocument", OuterXml, typeof(string));
+        }
+    }
+
+    [Serializable]
+    public class CombineStringsTask : IComputeTask<object, string, SerializableXmlDoc>
+    {
+        public IDictionary<IComputeJob<string>, IClusterNode> Map(IList<IClusterNode> subgrid, object arg)
+        {
+            return subgrid.ToDictionary(x => (IComputeJob<string>) new ToStringJob {Arg = arg}, x => x);
+        }
+
+        public ComputeJobResultPolicy Result(IComputeJobResult<string> res, IList<IComputeJobResult<string>> rcvd)
+        {
+            return ComputeJobResultPolicy.Wait;
+        }
+
+        public SerializableXmlDoc Reduce(IList<IComputeJobResult<string>> results)
+        {
+            var result = new SerializableXmlDoc();
+
+            result.LoadXml(CombineStrings(results.Select(x => x.Data())));
+
+            return result;
+        }
+
+        public static string CombineStrings(IEnumerable<string> strings)
+        {
+            var text = string.Concat(strings.Select(x => string.Format("<val>{0}</val>", x)));
+
+            return string.Format("<document>{0}</document>", text);
+        }
+    }
+
+    [Serializable]
+    public class ToStringJob : IComputeJob<string>
+    {
+        /// <summary>
+        /// Job argument.
+        /// </summary>
+        public object Arg { get; set; }
+
+        /** <inheritdoc /> */
+        public string Execute()
+        {
+            return Arg.ToString();
+        }
+
+        /** <inheritdoc /> */
+        public void Cancel()
+        {
+            // No-op.
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Services/ServiceProxyTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Services/ServiceProxyTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Services/ServiceProxyTest.cs
new file mode 100644
index 0000000..44e1d71
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Services/ServiceProxyTest.cs
@@ -0,0 +1,741 @@
+/*
+ * 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.Services
+{
+    using System;
+    using System.Diagnostics.CodeAnalysis;
+    using System.IO;
+    using System.Linq;
+    using System.Reflection;
+    using Apache.Ignite.Core.Impl.Memory;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Services;
+    using Apache.Ignite.Core.Portable;
+    using Apache.Ignite.Core.Services;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests <see cref="ServiceProxySerializer"/> functionality.
+    /// </summary>
+    public class ServiceProxyTest
+    {
+        /** */
+        private TestIgniteService _svc;
+
+        /** */
+        private readonly PortableMarshaller _marsh = new PortableMarshaller(new PortableConfiguration
+        {
+            TypeConfigurations = new[]
+            {
+                new PortableTypeConfiguration(typeof (TestPortableClass)),
+                new PortableTypeConfiguration(typeof (CustomExceptionPortable))
+            }
+        });
+
+        /** */
+        protected readonly IPortables Portables;
+
+        /** */
+        private readonly PlatformMemoryManager _memory = new PlatformMemoryManager(1024);
+
+        /** */
+        protected bool KeepPortable;
+
+        /** */
+        protected bool SrvKeepPortable;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ServiceProxyTest"/> class.
+        /// </summary>
+        public ServiceProxyTest()
+        {
+            Portables = new PortablesImpl(_marsh);
+        }
+
+        /// <summary>
+        /// Tests object class methods proxying.
+        /// </summary>
+        [Test]
+        public void TestObjectClassMethods()
+        {
+            var prx = GetProxy();
+
+            prx.IntProp = 12345;
+
+            Assert.AreEqual("12345", prx.ToString());
+            Assert.AreEqual("12345", _svc.ToString());
+            Assert.AreEqual(12345, prx.GetHashCode());
+            Assert.AreEqual(12345, _svc.GetHashCode());
+        }
+
+        /// <summary>
+        /// Tests properties proxying.
+        /// </summary>
+        [Test]
+        [SuppressMessage("ReSharper", "PossibleNullReferenceException")]
+        public void TestProperties()
+        {
+            var prx = GetProxy();
+
+            prx.IntProp = 10;
+            Assert.AreEqual(10, prx.IntProp);
+            Assert.AreEqual(10, _svc.IntProp);
+
+            _svc.IntProp = 15;
+            Assert.AreEqual(15, prx.IntProp);
+            Assert.AreEqual(15, _svc.IntProp);
+
+            prx.ObjProp = "prop1";
+            Assert.AreEqual("prop1", prx.ObjProp);
+            Assert.AreEqual("prop1", _svc.ObjProp);
+
+            prx.ObjProp = null;
+            Assert.IsNull(prx.ObjProp);
+            Assert.IsNull(_svc.ObjProp);
+
+            prx.ObjProp = new TestClass {Prop = "prop2"};
+            Assert.AreEqual("prop2", ((TestClass)prx.ObjProp).Prop);
+            Assert.AreEqual("prop2", ((TestClass)_svc.ObjProp).Prop);
+        }
+
+        /// <summary>
+        /// Tests void methods proxying.
+        /// </summary>
+        [Test]
+        public void TestVoidMethods()
+        {
+            var prx = GetProxy();
+
+            prx.VoidMethod();
+            Assert.AreEqual("VoidMethod", prx.InvokeResult);
+            Assert.AreEqual("VoidMethod", _svc.InvokeResult);
+
+            prx.VoidMethod(10);
+            Assert.AreEqual(_svc.InvokeResult, prx.InvokeResult);
+
+            prx.VoidMethod(10, "string");
+            Assert.AreEqual(_svc.InvokeResult, prx.InvokeResult);
+
+            prx.VoidMethod(10, "string", "arg");
+            Assert.AreEqual(_svc.InvokeResult, prx.InvokeResult);
+
+            prx.VoidMethod(10, "string", "arg", "arg1", 2, 3, "arg4");
+            Assert.AreEqual(_svc.InvokeResult, prx.InvokeResult);
+        }
+
+        /// <summary>
+        /// Tests object methods proxying.
+        /// </summary>
+        [Test]
+        public void TestObjectMethods()
+        {
+            var prx = GetProxy();
+
+            Assert.AreEqual("ObjectMethod", prx.ObjectMethod());
+            Assert.AreEqual("ObjectMethod987", prx.ObjectMethod(987));
+            Assert.AreEqual("ObjectMethod987str123", prx.ObjectMethod(987, "str123"));
+            Assert.AreEqual("ObjectMethod987str123TestClass", prx.ObjectMethod(987, "str123", new TestClass()));
+            Assert.AreEqual("ObjectMethod987str123TestClass34arg5arg6",
+                prx.ObjectMethod(987, "str123", new TestClass(), 3, 4, "arg5", "arg6"));
+        }
+
+        /// <summary>
+        /// Tests methods that exist in proxy interface, but do not exist in the actual service.
+        /// </summary>
+        [Test]
+        public void TestMissingMethods()
+        {
+            var prx = GetProxy();
+
+            var ex = Assert.Throws<InvalidOperationException>(() => prx.MissingMethod());
+
+            Assert.AreEqual("Failed to invoke proxy: there is no method 'MissingMethod'" +
+                            " in type 'Apache.Ignite.Core.Tests.Services.ServiceProxyTest+TestIgniteService'", ex.Message);
+        }
+
+        /// <summary>
+        /// Tests ambiguous methods handling (multiple methods with the same signature).
+        /// </summary>
+        [Test]
+        public void TestAmbiguousMethods()
+        {
+            var prx = GetProxy();
+
+            var ex = Assert.Throws<InvalidOperationException>(() => prx.AmbiguousMethod(1));
+
+            Assert.AreEqual("Failed to invoke proxy: there are 2 methods 'AmbiguousMethod' in type " +
+                            "'Apache.Ignite.Core.Tests.Services.ServiceProxyTest+TestIgniteService' with (Int32) arguments, " +
+                            "can't resolve ambiguity.", ex.Message);
+        }
+
+        [Test]
+        public void TestException()
+        {
+            var prx = GetProxy();
+
+            var err = Assert.Throws<ServiceInvocationException>(prx.ExceptionMethod);
+            Assert.AreEqual("Expected exception", err.InnerException.Message);
+
+            var ex = Assert.Throws<ServiceInvocationException>(() => prx.CustomExceptionMethod());
+            Assert.IsTrue(ex.ToString().Contains("+CustomException"));
+        }
+
+        [Test]
+        public void TestPortableMarshallingException()
+        {
+            var prx = GetProxy();
+                
+            var ex = Assert.Throws<ServiceInvocationException>(() => prx.CustomExceptionPortableMethod(false, false));
+
+            if (KeepPortable)
+            {
+                Assert.AreEqual("Proxy method invocation failed with a portable error. " +
+                                "Examine PortableCause for details.", ex.Message);
+
+                Assert.IsNotNull(ex.PortableCause);
+                Assert.IsNull(ex.InnerException);
+            }
+            else
+            {
+                Assert.AreEqual("Proxy method invocation failed with an exception. " +
+                                "Examine InnerException for details.", ex.Message);
+
+                Assert.IsNull(ex.PortableCause);
+                Assert.IsNotNull(ex.InnerException);
+            }
+
+            ex = Assert.Throws<ServiceInvocationException>(() => prx.CustomExceptionPortableMethod(true, false));
+            Assert.IsTrue(ex.ToString().Contains(
+                "Call completed with error, but error serialization failed [errType=CustomExceptionPortable, " +
+                "serializationErrMsg=Expected exception in CustomExceptionPortable.WritePortable]"));
+
+            ex = Assert.Throws<ServiceInvocationException>(() => prx.CustomExceptionPortableMethod(true, true));
+            Assert.IsTrue(ex.ToString().Contains(
+                "Call completed with error, but error serialization failed [errType=CustomExceptionPortable, " +
+                "serializationErrMsg=Expected exception in CustomExceptionPortable.WritePortable]"));
+        }
+
+        /// <summary>
+        /// Creates the proxy.
+        /// </summary>
+        protected ITestIgniteServiceProxyInterface GetProxy()
+        {
+            return GetProxy<ITestIgniteServiceProxyInterface>();
+        }
+
+        /// <summary>
+        /// Creates the proxy.
+        /// </summary>
+        protected T GetProxy<T>()
+        {
+            _svc = new TestIgniteService(Portables);
+
+            var prx = new ServiceProxy<T>(InvokeProxyMethod).GetTransparentProxy();
+
+            Assert.IsFalse(ReferenceEquals(_svc, prx));
+
+            return prx;
+        }
+
+        /// <summary>
+        /// Invokes the proxy.
+        /// </summary>
+        /// <param name="method">Method.</param>
+        /// <param name="args">Arguments.</param>
+        /// <returns>
+        /// Invocation result.
+        /// </returns>
+        private object InvokeProxyMethod(MethodBase method, object[] args)
+        {
+            using (var inStream = new PlatformMemoryStream(_memory.Allocate()))
+            using (var outStream = new PlatformMemoryStream(_memory.Allocate()))
+            {
+                // 1) Write to a stream
+                inStream.WriteBool(SrvKeepPortable);  // WriteProxyMethod does not do this, but Java does
+
+                ServiceProxySerializer.WriteProxyMethod(_marsh.StartMarshal(inStream), method, args);
+
+                inStream.SynchronizeOutput();
+
+                inStream.Seek(0, SeekOrigin.Begin);
+
+                // 2) call InvokeServiceMethod
+                string mthdName;
+                object[] mthdArgs;
+
+                ServiceProxySerializer.ReadProxyMethod(inStream, _marsh, out mthdName, out mthdArgs);
+
+                var result = ServiceProxyInvoker.InvokeServiceMethod(_svc, mthdName, mthdArgs);
+
+                ServiceProxySerializer.WriteInvocationResult(outStream, _marsh, result.Key, result.Value);
+                
+                _marsh.StartMarshal(outStream).WriteString("unused");  // fake Java exception details
+
+                outStream.SynchronizeOutput();
+
+                outStream.Seek(0, SeekOrigin.Begin);
+
+                return ServiceProxySerializer.ReadInvocationResult(outStream, _marsh, KeepPortable);
+            }
+        }
+
+        /// <summary>
+        /// Test service interface.
+        /// </summary>
+        protected interface ITestIgniteServiceProperties
+        {
+            /** */
+            int IntProp { get; set; }
+
+            /** */
+            object ObjProp { get; set; }
+
+            /** */
+            string InvokeResult { get; }
+        }
+
+        /// <summary>
+        /// Test service interface to check ambiguity handling.
+        /// </summary>
+        protected interface ITestIgniteServiceAmbiguity
+        {
+            /** */
+            int AmbiguousMethod(int arg);
+        }
+
+        /// <summary>
+        /// Test service interface.
+        /// </summary>
+        protected interface ITestIgniteService : ITestIgniteServiceProperties
+        {
+            /** */
+            void VoidMethod();
+
+            /** */
+            void VoidMethod(int arg);
+
+            /** */
+            void VoidMethod(int arg, string arg1, object arg2 = null);
+
+            /** */
+            void VoidMethod(int arg, string arg1, object arg2 = null, params object[] args);
+
+            /** */
+            object ObjectMethod();
+
+            /** */
+            object ObjectMethod(int arg);
+
+            /** */
+            object ObjectMethod(int arg, string arg1, object arg2 = null);
+
+            /** */
+            object ObjectMethod(int arg, string arg1, object arg2 = null, params object[] args);
+
+            /** */
+            void ExceptionMethod();
+
+            /** */
+            void CustomExceptionMethod();
+
+            /** */
+            void CustomExceptionPortableMethod(bool throwOnWrite, bool throwOnRead);
+
+            /** */
+            TestPortableClass PortableArgMethod(int arg1, IPortableObject arg2);
+
+            /** */
+            IPortableObject PortableResultMethod(int arg1, TestPortableClass arg2);
+
+            /** */
+            IPortableObject PortableArgAndResultMethod(int arg1, IPortableObject arg2);
+
+            /** */
+            int AmbiguousMethod(int arg);
+        }
+
+        /// <summary>
+        /// Test service interface. Does not derive from actual interface, but has all the same method signatures.
+        /// </summary>
+        protected interface ITestIgniteServiceProxyInterface
+        {
+            /** */
+            int IntProp { get; set; }
+
+            /** */
+            object ObjProp { get; set; }
+
+            /** */
+            string InvokeResult { get; }
+
+            /** */
+            void VoidMethod();
+
+            /** */
+            void VoidMethod(int arg);
+
+            /** */
+            void VoidMethod(int arg, string arg1, object arg2 = null);
+
+            /** */
+            void VoidMethod(int arg, string arg1, object arg2 = null, params object[] args);
+
+            /** */
+            object ObjectMethod();
+
+            /** */
+            object ObjectMethod(int arg);
+
+            /** */
+            object ObjectMethod(int arg, string arg1, object arg2 = null);
+
+            /** */
+            object ObjectMethod(int arg, string arg1, object arg2 = null, params object[] args);
+
+            /** */
+            void ExceptionMethod();
+
+            /** */
+            void CustomExceptionMethod();
+
+            /** */
+            void CustomExceptionPortableMethod(bool throwOnWrite, bool throwOnRead);
+
+            /** */
+            TestPortableClass PortableArgMethod(int arg1, IPortableObject arg2);
+
+            /** */
+            IPortableObject PortableResultMethod(int arg1, TestPortableClass arg2);
+
+            /** */
+            IPortableObject PortableArgAndResultMethod(int arg1, IPortableObject arg2);
+
+            /** */
+            void MissingMethod();
+
+            /** */
+            int AmbiguousMethod(int arg);
+        }
+
+        /// <summary>
+        /// Test service.
+        /// </summary>
+        [Serializable]
+        private class TestIgniteService : ITestIgniteService, ITestIgniteServiceAmbiguity
+        {
+            /** */
+            private readonly IPortables _portables;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="TestIgniteService"/> class.
+            /// </summary>
+            /// <param name="portables">The portables.</param>
+            public TestIgniteService(IPortables portables)
+            {
+                _portables = portables;
+            }
+
+            /** <inheritdoc /> */
+            public int IntProp { get; set; }
+
+            /** <inheritdoc /> */
+            public object ObjProp { get; set; }
+
+            /** <inheritdoc /> */
+            public string InvokeResult { get; private set; }
+
+            /** <inheritdoc /> */
+            public void VoidMethod()
+            {
+                InvokeResult = "VoidMethod";
+            }
+
+            /** <inheritdoc /> */
+            public void VoidMethod(int arg)
+            {
+                InvokeResult = "VoidMethod" + arg;
+            }
+
+            /** <inheritdoc /> */
+            public void VoidMethod(int arg, string arg1, object arg2 = null)
+            {
+                InvokeResult = "VoidMethod" + arg + arg1 + arg2;
+            }
+
+            /** <inheritdoc /> */
+            public void VoidMethod(int arg, string arg1, object arg2 = null, params object[] args)
+            {
+                InvokeResult = "VoidMethod" + arg + arg1 + arg2 + string.Concat(args.Select(x => x.ToString()));
+            }
+
+            /** <inheritdoc /> */
+            public object ObjectMethod()
+            {
+                return "ObjectMethod";
+            }
+
+            /** <inheritdoc /> */
+            public object ObjectMethod(int arg)
+            {
+                return "ObjectMethod" + arg;
+            }
+
+            /** <inheritdoc /> */
+            public object ObjectMethod(int arg, string arg1, object arg2 = null)
+            {
+                return "ObjectMethod" + arg + arg1 + arg2;
+            }
+
+            /** <inheritdoc /> */
+            public object ObjectMethod(int arg, string arg1, object arg2 = null, params object[] args)
+            {
+                return "ObjectMethod" + arg + arg1 + arg2 + string.Concat(args.Select(x => x.ToString()));
+            }
+
+            /** <inheritdoc /> */
+            public void ExceptionMethod()
+            {
+                throw new ArithmeticException("Expected exception");
+            }
+
+            /** <inheritdoc /> */
+            public void CustomExceptionMethod()
+            {
+                throw new CustomException();
+            }
+
+            /** <inheritdoc /> */
+            public void CustomExceptionPortableMethod(bool throwOnWrite, bool throwOnRead)
+            {
+                throw new CustomExceptionPortable {ThrowOnRead = throwOnRead, ThrowOnWrite = throwOnWrite};
+            }
+
+            /** <inheritdoc /> */
+            public TestPortableClass PortableArgMethod(int arg1, IPortableObject arg2)
+            {
+                return arg2.Deserialize<TestPortableClass>();
+            }
+
+            /** <inheritdoc /> */
+            public IPortableObject PortableResultMethod(int arg1, TestPortableClass arg2)
+            {
+                return _portables.ToPortable<IPortableObject>(arg2);
+            }
+
+            /** <inheritdoc /> */
+            public IPortableObject PortableArgAndResultMethod(int arg1, IPortableObject arg2)
+            {
+                return _portables.ToPortable<IPortableObject>(arg2.Deserialize<TestPortableClass>());
+            }
+
+            /** <inheritdoc /> */
+            public override string ToString()
+            {
+                return IntProp.ToString();
+            }
+
+            /** <inheritdoc /> */
+            public override int GetHashCode()
+            {
+                return IntProp.GetHashCode();
+            }
+
+            /** <inheritdoc /> */
+            int ITestIgniteService.AmbiguousMethod(int arg)
+            {
+                return arg;
+            }
+
+            /** <inheritdoc /> */
+            int ITestIgniteServiceAmbiguity.AmbiguousMethod(int arg)
+            {
+                return -arg;
+            }
+        }
+
+        /// <summary>
+        /// Test serializable class.
+        /// </summary>
+        [Serializable]
+        private class TestClass
+        {
+            /** */
+            public string Prop { get; set; }
+
+            /** <inheritdoc /> */
+            public override string ToString()
+            {
+                return "TestClass" + Prop;
+            }
+        }
+
+        /// <summary>
+        /// Custom non-serializable exception.
+        /// </summary>
+        private class CustomException : Exception
+        {
+            
+        }
+
+        /// <summary>
+        /// Custom non-serializable exception.
+        /// </summary>
+        private class CustomExceptionPortable : Exception, IPortableMarshalAware
+        {
+            /** */
+            public bool ThrowOnWrite { get; set; }
+
+            /** */
+            public bool ThrowOnRead { get; set; }
+
+            /** <inheritdoc /> */
+            public void WritePortable(IPortableWriter writer)
+            {
+                writer.WriteBoolean("ThrowOnRead", ThrowOnRead);
+
+                if (ThrowOnWrite)
+                    throw new Exception("Expected exception in CustomExceptionPortable.WritePortable");
+            }
+
+            /** <inheritdoc /> */
+            public void ReadPortable(IPortableReader reader)
+            {
+                ThrowOnRead = reader.ReadBoolean("ThrowOnRead");
+
+                if (ThrowOnRead)
+                    throw new Exception("Expected exception in CustomExceptionPortable.ReadPortable");
+            }
+        }
+
+        /// <summary>
+        /// Portable object for method argument/result.
+        /// </summary>
+        protected class TestPortableClass : IPortableMarshalAware
+        {
+            /** */
+            public string Prop { get; set; }
+
+            /** */
+            public bool ThrowOnWrite { get; set; }
+
+            /** */
+            public bool ThrowOnRead { get; set; }
+
+            /** <inheritdoc /> */
+            public void WritePortable(IPortableWriter writer)
+            {
+                writer.WriteString("Prop", Prop);
+                writer.WriteBoolean("ThrowOnRead", ThrowOnRead);
+
+                if (ThrowOnWrite)
+                    throw new Exception("Expected exception in TestPortableClass.WritePortable");
+            }
+
+            /** <inheritdoc /> */
+            public void ReadPortable(IPortableReader reader)
+            {
+                Prop = reader.ReadString("Prop");
+                ThrowOnRead = reader.ReadBoolean("ThrowOnRead");
+
+                if (ThrowOnRead)
+                    throw new Exception("Expected exception in TestPortableClass.ReadPortable");
+            }
+        }
+    }
+
+    /// <summary>
+    /// Tests <see cref="ServiceProxySerializer"/> functionality with keepPortable mode enabled on client.
+    /// </summary>
+    public class ServiceProxyTestKeepPortableClient : ServiceProxyTest
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ServiceProxyTestKeepPortableClient"/> class.
+        /// </summary>
+        public ServiceProxyTestKeepPortableClient()
+        {
+            KeepPortable = true;
+        }
+
+        [Test]
+        public void TestPortableMethods()
+        {
+            var prx = GetProxy();
+
+            var obj = new TestPortableClass { Prop = "PropValue" };
+
+            var result = prx.PortableResultMethod(1, obj);
+
+            Assert.AreEqual(obj.Prop, result.Deserialize<TestPortableClass>().Prop);
+        }
+    }
+
+    /// <summary>
+    /// Tests <see cref="ServiceProxySerializer"/> functionality with keepPortable mode enabled on server.
+    /// </summary>
+    public class ServiceProxyTestKeepPortableServer : ServiceProxyTest
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ServiceProxyTestKeepPortableServer"/> class.
+        /// </summary>
+        public ServiceProxyTestKeepPortableServer()
+        {
+            SrvKeepPortable = true;
+        }
+
+        [Test]
+        public void TestPortableMethods()
+        {
+            var prx = GetProxy();
+
+            var obj = new TestPortableClass { Prop = "PropValue" };
+            var portObj = Portables.ToPortable<IPortableObject>(obj);
+
+            var result = prx.PortableArgMethod(1, portObj);
+
+            Assert.AreEqual(obj.Prop, result.Prop);
+        }
+    }
+
+    /// <summary>
+    /// Tests <see cref="ServiceProxySerializer"/> functionality with keepPortable mode enabled on client and on server.
+    /// </summary>
+    public class ServiceProxyTestKeepPortableClientServer : ServiceProxyTest
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ServiceProxyTestKeepPortableClientServer"/> class.
+        /// </summary>
+        public ServiceProxyTestKeepPortableClientServer()
+        {
+            KeepPortable = true;
+            SrvKeepPortable = true;
+        }
+
+        [Test]
+        public void TestPortableMethods()
+        {
+            var prx = GetProxy();
+            
+            var obj = new TestPortableClass { Prop = "PropValue" };
+            var portObj = Portables.ToPortable<IPortableObject>(obj);
+
+            var result = prx.PortableArgAndResultMethod(1, portObj);
+
+            Assert.AreEqual(obj.Prop, result.Deserialize<TestPortableClass>().Prop);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Services/ServicesAsyncWrapper.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Services/ServicesAsyncWrapper.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Services/ServicesAsyncWrapper.cs
new file mode 100644
index 0000000..ba45dbd
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Services/ServicesAsyncWrapper.cs
@@ -0,0 +1,174 @@
+/*
+ * 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.Services
+{
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Services;
+
+    /// <summary>
+    /// Services async wrapper to simplify testing.
+    /// </summary>
+    public class ServicesAsyncWrapper : IServices
+    {
+        /** Wrapped async services. */
+        private readonly IServices _services;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ServicesAsyncWrapper"/> class.
+        /// </summary>
+        /// <param name="services">Services to wrap.</param>
+        public ServicesAsyncWrapper(IServices services)
+        {
+            _services = services.WithAsync();
+        }
+
+        /** <inheritDoc /> */
+        public IServices WithAsync()
+        {
+            return this;
+        }
+
+        /** <inheritDoc /> */
+        public bool IsAsync
+        {
+            get { return true; }
+        }
+
+        /** <inheritDoc /> */
+        public IFuture GetFuture()
+        {
+            Debug.Fail("ServicesAsyncWrapper.Future() should not be called. It always returns null.");
+            return null;
+        }
+
+        /** <inheritDoc /> */
+        public IFuture<TResult> GetFuture<TResult>()
+        {
+            Debug.Fail("ServicesAsyncWrapper.Future() should not be called. It always returns null.");
+            return null;
+        }
+
+        /** <inheritDoc /> */
+        public IClusterGroup ClusterGroup
+        {
+            get { return _services.ClusterGroup; }
+        }
+
+        /** <inheritDoc /> */
+        public void DeployClusterSingleton(string name, IService service)
+        {
+            _services.DeployClusterSingleton(name, service);
+            WaitResult();
+        }
+
+        /** <inheritDoc /> */
+        public void DeployNodeSingleton(string name, IService service)
+        {
+            _services.DeployNodeSingleton(name, service);
+            WaitResult();
+        }
+
+        /** <inheritDoc /> */
+        public void DeployKeyAffinitySingleton<TK>(string name, IService service, string cacheName, TK affinityKey)
+        {
+            _services.DeployKeyAffinitySingleton(name, service, cacheName, affinityKey);
+            WaitResult();
+        }
+
+        /** <inheritDoc /> */
+        public void DeployMultiple(string name, IService service, int totalCount, int maxPerNodeCount)
+        {
+            _services.DeployMultiple(name, service, totalCount, maxPerNodeCount);
+            WaitResult();
+        }
+
+        /** <inheritDoc /> */
+        public void Deploy(ServiceConfiguration configuration)
+        {
+            _services.Deploy(configuration);
+            WaitResult();
+        }
+
+        /** <inheritDoc /> */
+        public void Cancel(string name)
+        {
+            _services.Cancel(name);
+            WaitResult();
+        }
+
+        /** <inheritDoc /> */
+        public void CancelAll()
+        {
+            _services.CancelAll();
+            WaitResult();
+        }
+
+        /** <inheritDoc /> */
+        public ICollection<IServiceDescriptor> GetServiceDescriptors()
+        {
+            return _services.GetServiceDescriptors();
+        }
+
+        /** <inheritDoc /> */
+        public T GetService<T>(string name)
+        {
+            return _services.GetService<T>(name);
+        }
+
+        /** <inheritDoc /> */
+        public ICollection<T> GetServices<T>(string name)
+        {
+            return _services.GetServices<T>(name);
+        }
+
+        /** <inheritDoc /> */
+        public T GetServiceProxy<T>(string name) where T : class
+        {
+            return _services.GetServiceProxy<T>(name);
+        }
+
+        /** <inheritDoc /> */
+        public T GetServiceProxy<T>(string name, bool sticky) where T : class
+        {
+            return _services.GetServiceProxy<T>(name, sticky);
+        }
+
+        /** <inheritDoc /> */
+        public IServices WithKeepPortable()
+        {
+            return new ServicesAsyncWrapper(_services.WithKeepPortable());
+        }
+
+        /** <inheritDoc /> */
+        public IServices WithServerKeepPortable()
+        {
+            return new ServicesAsyncWrapper(_services.WithServerKeepPortable());
+        }
+
+        /// <summary>
+        /// Waits for the async result.
+        /// </summary>
+        private void WaitResult()
+        {
+            _services.GetFuture().Get();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
new file mode 100644
index 0000000..7f5aa44
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
@@ -0,0 +1,823 @@
+/*
+ * 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.Services
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Linq;
+    using System.Threading;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Portable;
+    using Apache.Ignite.Core.Resource;
+    using Apache.Ignite.Core.Services;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Services tests.
+    /// </summary>
+    public class ServicesTest
+    {
+        /** */
+        private const string SvcName = "Service1";
+
+        /** */
+        private const string CacheName = "cache1";
+
+        /** */
+        private const int AffKey = 25;
+
+        /** */
+        protected IIgnite Grid1;
+
+        /** */
+        protected IIgnite Grid2;
+
+        /** */
+        protected IIgnite Grid3;
+
+        /** */
+        protected IIgnite[] Grids;
+
+        [TestFixtureTearDown]
+        public void FixtureTearDown()
+        {
+            StopGrids();
+        }
+
+        /// <summary>
+        /// Executes before each test.
+        /// </summary>
+        [SetUp]
+        public void SetUp()
+        {
+            StartGrids();
+            EventsTestHelper.ListenResult = true;
+        }
+
+        /// <summary>
+        /// Executes after each test.
+        /// </summary>
+        [TearDown]
+        public void TearDown()
+        {
+            try
+            {
+                Services.Cancel(SvcName);
+
+                TestUtils.AssertHandleRegistryIsEmpty(1000, Grid1, Grid2, Grid3);
+            }
+            catch (Exception)
+            {
+                // Restart grids to cleanup
+                StopGrids();
+
+                throw;
+            }
+            finally
+            {
+                EventsTestHelper.AssertFailures();
+
+                if (TestContext.CurrentContext.Test.Name.StartsWith("TestEventTypes"))
+                    StopGrids(); // clean events for other tests
+            }
+        }
+
+        /// <summary>
+        /// Tests deployment.
+        /// </summary>
+        [Test]
+        public void TestDeploy([Values(true, false)] bool portable)
+        {
+            var cfg = new ServiceConfiguration
+            {
+                Name = SvcName,
+                MaxPerNodeCount = 3,
+                TotalCount = 3,
+                NodeFilter = new NodeFilter {NodeId = Grid1.Cluster.LocalNode.Id},
+                Service = portable ? new TestIgniteServicePortable() : new TestIgniteServiceSerializable()
+            };
+
+            Services.Deploy(cfg);
+
+            CheckServiceStarted(Grid1, 3);
+        }
+
+        /// <summary>
+        /// Tests cluster singleton deployment.
+        /// </summary>
+        [Test]
+        public void TestDeployClusterSingleton()
+        {
+            var svc = new TestIgniteServiceSerializable();
+
+            Services.DeployClusterSingleton(SvcName, svc);
+
+            var svc0 = Services.GetServiceProxy<ITestIgniteService>(SvcName);
+
+            // Check that only one node has the service.
+            foreach (var grid in Grids)
+            {
+                if (grid.Cluster.LocalNode.Id == svc0.NodeId)
+                    CheckServiceStarted(grid);
+                else
+                    Assert.IsNull(grid.Services().GetService<TestIgniteServiceSerializable>(SvcName));
+            }
+        }
+
+        /// <summary>
+        /// Tests node singleton deployment.
+        /// </summary>
+        [Test]
+        public void TestDeployNodeSingleton()
+        {
+            var svc = new TestIgniteServiceSerializable();
+
+            Services.DeployNodeSingleton(SvcName, svc);
+
+            Assert.AreEqual(1, Grid1.Services().GetServices<ITestIgniteService>(SvcName).Count);
+            Assert.AreEqual(1, Grid2.Services().GetServices<ITestIgniteService>(SvcName).Count);
+            Assert.AreEqual(1, Grid3.Services().GetServices<ITestIgniteService>(SvcName).Count);
+        }
+
+        /// <summary>
+        /// Tests key affinity singleton deployment.
+        /// </summary>
+        [Test]
+        public void TestDeployKeyAffinitySingleton()
+        {
+            var svc = new TestIgniteServicePortable();
+
+            Services.DeployKeyAffinitySingleton(SvcName, svc, CacheName, AffKey);
+
+            var affNode = Grid1.Affinity(CacheName).MapKeyToNode(AffKey);
+
+            var prx = Services.GetServiceProxy<ITestIgniteService>(SvcName);
+
+            Assert.AreEqual(affNode.Id, prx.NodeId);
+        }
+
+        /// <summary>
+        /// Tests key affinity singleton deployment.
+        /// </summary>
+        [Test]
+        public void TestDeployKeyAffinitySingletonPortable()
+        {
+            var services = Services.WithKeepPortable();
+
+            var svc = new TestIgniteServicePortable();
+
+            var affKey = new PortableObject {Val = AffKey};
+
+            services.DeployKeyAffinitySingleton(SvcName, svc, CacheName, affKey);
+
+            var prx = services.GetServiceProxy<ITestIgniteService>(SvcName);
+
+            Assert.IsTrue(prx.Initialized);
+        }
+
+        /// <summary>
+        /// Tests multiple deployment.
+        /// </summary>
+        [Test]
+        public void TestDeployMultiple()
+        {
+            var svc = new TestIgniteServiceSerializable();
+
+            Services.DeployMultiple(SvcName, svc, Grids.Length * 5, 5);
+
+            foreach (var grid in Grids)
+                CheckServiceStarted(grid, 5);
+        }
+
+        /// <summary>
+        /// Tests cancellation.
+        /// </summary>
+        [Test]
+        public void TestCancel()
+        {
+            for (var i = 0; i < 10; i++)
+            {
+                Services.DeployNodeSingleton(SvcName + i, new TestIgniteServicePortable());
+                Assert.IsNotNull(Services.GetService<ITestIgniteService>(SvcName + i));
+            }
+
+            Services.Cancel(SvcName + 0);
+            Services.Cancel(SvcName + 1);
+
+            Assert.IsNull(Services.GetService<ITestIgniteService>(SvcName + 0));
+            Assert.IsNull(Services.GetService<ITestIgniteService>(SvcName + 1));
+
+            for (var i = 2; i < 10; i++)
+                Assert.IsNotNull(Services.GetService<ITestIgniteService>(SvcName + i));
+
+            Services.CancelAll();
+
+            for (var i = 0; i < 10; i++)
+                Assert.IsNull(Services.GetService<ITestIgniteService>(SvcName + i));
+        }
+
+        /// <summary>
+        /// Tests service proxy.
+        /// </summary>
+        [Test]
+        public void TestGetServiceProxy([Values(true, false)] bool portable)
+        {
+            // Test proxy without a service
+            var prx = Services.GetServiceProxy<ITestIgniteService>(SvcName);
+
+            Assert.IsTrue(prx != null);
+
+            var ex = Assert.Throws<ServiceInvocationException>(() => Assert.IsTrue(prx.Initialized)).InnerException;
+            Assert.AreEqual("Failed to find deployed service: " + SvcName, ex.Message);
+
+            // Deploy to grid2 & grid3
+            var svc = portable
+                ? new TestIgniteServicePortable {TestProperty = 17}
+                : new TestIgniteServiceSerializable {TestProperty = 17};
+
+            Grid1.Cluster.ForNodeIds(Grid2.Cluster.LocalNode.Id, Grid3.Cluster.LocalNode.Id).Services()
+                .DeployNodeSingleton(SvcName,
+                    svc);
+
+            // Make sure there is no local instance on grid1
+            Assert.IsNull(Services.GetService<ITestIgniteService>(SvcName));
+
+            // Get proxy
+            prx = Services.GetServiceProxy<ITestIgniteService>(SvcName);
+
+            // Check proxy properties
+            Assert.IsNotNull(prx);
+            Assert.AreEqual(prx.GetType(), svc.GetType());
+            Assert.AreEqual(prx.ToString(), svc.ToString());
+            Assert.AreEqual(17, prx.TestProperty);
+            Assert.IsTrue(prx.Initialized);
+            Assert.IsTrue(prx.Executed);
+            Assert.IsFalse(prx.Cancelled);
+            Assert.AreEqual(SvcName, prx.LastCallContextName);
+
+            // Check err method
+            Assert.Throws<ServiceInvocationException>(() => prx.ErrMethod(123));
+
+            // Check local scenario (proxy should not be created for local instance)
+            Assert.IsTrue(ReferenceEquals(Grid2.Services().GetService<ITestIgniteService>(SvcName),
+                Grid2.Services().GetServiceProxy<ITestIgniteService>(SvcName)));
+
+            // Check sticky = false: call multiple times, check that different nodes get invoked
+            var invokedIds = Enumerable.Range(1, 100).Select(x => prx.NodeId).Distinct().ToList();
+            Assert.AreEqual(2, invokedIds.Count);
+
+            // Check sticky = true: all calls should be to the same node
+            prx = Services.GetServiceProxy<ITestIgniteService>(SvcName, true);
+            invokedIds = Enumerable.Range(1, 100).Select(x => prx.NodeId).Distinct().ToList();
+            Assert.AreEqual(1, invokedIds.Count);
+
+            // Proxy does not work for cancelled service.
+            Services.CancelAll();
+
+            Assert.Throws<ServiceInvocationException>(() => { Assert.IsTrue(prx.Cancelled); });
+        }
+
+        /// <summary>
+        /// Tests the duck typing: proxy interface can be different from actual service interface, 
+        /// only called method signature should be compatible.
+        /// </summary>
+        [Test]
+        public void TestDuckTyping([Values(true, false)] bool local)
+        {
+            var svc = new TestIgniteServicePortable {TestProperty = 33};
+
+            // Deploy locally or to the remote node
+            var nodeId = (local ? Grid1 : Grid2).Cluster.LocalNode.Id;
+            
+            var cluster = Grid1.Cluster.ForNodeIds(nodeId);
+
+            cluster.Services().DeployNodeSingleton(SvcName, svc);
+
+            // Get proxy
+            var prx = Services.GetServiceProxy<ITestIgniteServiceProxyInterface>(SvcName);
+
+            // NodeId signature is the same as in service
+            Assert.AreEqual(nodeId, prx.NodeId);
+            
+            // Method signature is different from service signature (object -> object), but is compatible.
+            Assert.AreEqual(15, prx.Method(15));
+
+            // TestProperty is object in proxy and int in service, getter works..
+            Assert.AreEqual(33, prx.TestProperty);
+
+            // .. but setter does not
+            var ex = Assert.Throws<ServiceInvocationException>(() => { prx.TestProperty = new object(); });
+            Assert.AreEqual("Object of type 'System.Object' cannot be converted to type 'System.Int32'.",
+                ex.InnerException.Message);
+        }
+
+        /// <summary>
+        /// Tests service descriptors.
+        /// </summary>
+        [Test]
+        public void TestServiceDescriptors()
+        {
+            Services.DeployKeyAffinitySingleton(SvcName, new TestIgniteServiceSerializable(), CacheName, 1);
+
+            var descriptors = Services.GetServiceDescriptors();
+
+            Assert.AreEqual(1, descriptors.Count);
+
+            var desc = descriptors.Single();
+
+            Assert.AreEqual(SvcName, desc.Name);
+            Assert.AreEqual(CacheName, desc.CacheName);
+            Assert.AreEqual(1, desc.AffinityKey);
+            Assert.AreEqual(1, desc.MaxPerNodeCount);
+            Assert.AreEqual(1, desc.TotalCount);
+            Assert.AreEqual(typeof(TestIgniteServiceSerializable), desc.Type);
+            Assert.AreEqual(Grid1.Cluster.LocalNode.Id, desc.OriginNodeId);
+
+            var top = desc.TopologySnapshot;
+            var prx = Services.GetServiceProxy<ITestIgniteService>(SvcName);
+            
+            Assert.AreEqual(1, top.Count);
+            Assert.AreEqual(prx.NodeId, top.Keys.Single());
+            Assert.AreEqual(1, top.Values.Single());
+        }
+
+        /// <summary>
+        /// Tests the client portable flag.
+        /// </summary>
+        [Test]
+        public void TestWithKeepPortableClient()
+        {
+            var svc = new TestIgniteServicePortable();
+
+            // Deploy to grid2
+            Grid1.Cluster.ForNodeIds(Grid2.Cluster.LocalNode.Id).Services().WithKeepPortable()
+                .DeployNodeSingleton(SvcName, svc);
+
+            // Get proxy
+            var prx = Services.WithKeepPortable().GetServiceProxy<ITestIgniteService>(SvcName);
+
+            var obj = new PortableObject {Val = 11};
+
+            var res = (IPortableObject) prx.Method(obj);
+            Assert.AreEqual(11, res.Deserialize<PortableObject>().Val);
+
+            res = (IPortableObject) prx.Method(Grid1.Portables().ToPortable<IPortableObject>(obj));
+            Assert.AreEqual(11, res.Deserialize<PortableObject>().Val);
+        }
+        
+        /// <summary>
+        /// Tests the server portable flag.
+        /// </summary>
+        [Test]
+        public void TestWithKeepPortableServer()
+        {
+            var svc = new TestIgniteServicePortable();
+
+            // Deploy to grid2
+            Grid1.Cluster.ForNodeIds(Grid2.Cluster.LocalNode.Id).Services().WithServerKeepPortable()
+                .DeployNodeSingleton(SvcName, svc);
+
+            // Get proxy
+            var prx = Services.WithServerKeepPortable().GetServiceProxy<ITestIgniteService>(SvcName);
+
+            var obj = new PortableObject { Val = 11 };
+
+            var res = (PortableObject) prx.Method(obj);
+            Assert.AreEqual(11, res.Val);
+
+            res = (PortableObject)prx.Method(Grid1.Portables().ToPortable<IPortableObject>(obj));
+            Assert.AreEqual(11, res.Val);
+        }
+
+        /// <summary>
+        /// Tests server and client portable flag.
+        /// </summary>
+        [Test]
+        public void TestWithKeepPortableBoth()
+        {
+            var svc = new TestIgniteServicePortable();
+
+            // Deploy to grid2
+            Grid1.Cluster.ForNodeIds(Grid2.Cluster.LocalNode.Id).Services().WithKeepPortable().WithServerKeepPortable()
+                .DeployNodeSingleton(SvcName, svc);
+
+            // Get proxy
+            var prx = Services.WithKeepPortable().WithServerKeepPortable().GetServiceProxy<ITestIgniteService>(SvcName);
+
+            var obj = new PortableObject { Val = 11 };
+
+            var res = (IPortableObject)prx.Method(obj);
+            Assert.AreEqual(11, res.Deserialize<PortableObject>().Val);
+
+            res = (IPortableObject)prx.Method(Grid1.Portables().ToPortable<IPortableObject>(obj));
+            Assert.AreEqual(11, res.Deserialize<PortableObject>().Val);
+        }
+
+        /// <summary>
+        /// Tests exception in Initialize.
+        /// </summary>
+        [Test]
+        public void TestInitException()
+        {
+            var svc = new TestIgniteServiceSerializable { ThrowInit = true };
+
+            var ex = Assert.Throws<IgniteException>(() => Services.DeployMultiple(SvcName, svc, Grids.Length, 1));
+            Assert.AreEqual("Expected exception", ex.Message);
+
+            var svc0 = Services.GetService<TestIgniteServiceSerializable>(SvcName);
+
+            Assert.IsNull(svc0);
+        }
+
+        /// <summary>
+        /// Tests exception in Execute.
+        /// </summary>
+        [Test]
+        public void TestExecuteException()
+        {
+            var svc = new TestIgniteServiceSerializable { ThrowExecute = true };
+
+            Services.DeployMultiple(SvcName, svc, Grids.Length, 1);
+
+            var svc0 = Services.GetService<TestIgniteServiceSerializable>(SvcName);
+
+            // Execution failed, but service exists.
+            Assert.IsNotNull(svc0);
+            Assert.IsFalse(svc0.Executed);
+        }
+
+        /// <summary>
+        /// Tests exception in Cancel.
+        /// </summary>
+        [Test]
+        public void TestCancelException()
+        {
+            var svc = new TestIgniteServiceSerializable { ThrowCancel = true };
+
+            Services.DeployMultiple(SvcName, svc, Grids.Length, 1);
+
+            CheckServiceStarted(Grid1);
+
+            Services.CancelAll();
+
+            // Cancellation failed, but service is removed.
+            foreach (var grid in Grids)
+                Assert.IsNull(grid.Services().GetService<ITestIgniteService>(SvcName));
+        }
+
+        [Test]
+        public void TestMarshalExceptionOnRead()
+        {
+            var svc = new TestIgniteServicePortableErr();
+
+            var ex = Assert.Throws<IgniteException>(() => Services.DeployMultiple(SvcName, svc, Grids.Length, 1));
+            Assert.AreEqual("Expected exception", ex.Message);
+
+            var svc0 = Services.GetService<TestIgniteServiceSerializable>(SvcName);
+
+            Assert.IsNull(svc0);
+        }
+
+        [Test]
+        public void TestMarshalExceptionOnWrite()
+        {
+            var svc = new TestIgniteServicePortableErr {ThrowOnWrite = true};
+
+            var ex = Assert.Throws<Exception>(() => Services.DeployMultiple(SvcName, svc, Grids.Length, 1));
+            Assert.AreEqual("Expected exception", ex.Message);
+
+            var svc0 = Services.GetService<TestIgniteServiceSerializable>(SvcName);
+
+            Assert.IsNull(svc0);
+        }
+
+        /// <summary>
+        /// Starts the grids.
+        /// </summary>
+        private void StartGrids()
+        {
+            if (Grid1 != null)
+                return;
+
+            Grid1 = Ignition.Start(Configuration("config\\compute\\compute-grid1.xml"));
+            Grid2 = Ignition.Start(Configuration("config\\compute\\compute-grid2.xml"));
+            Grid3 = Ignition.Start(Configuration("config\\compute\\compute-grid3.xml"));
+
+            Grids = new[] { Grid1, Grid2, Grid3 };
+        }
+
+        /// <summary>
+        /// Stops the grids.
+        /// </summary>
+        private void StopGrids()
+        {
+            Grid1 = Grid2 = Grid3 = null;
+            Grids = null;
+
+            Ignition.StopAll(true);
+        }
+
+        /// <summary>
+        /// Checks that service has started on specified grid.
+        /// </summary>
+        private static void CheckServiceStarted(IIgnite grid, int count = 1)
+        {
+            var services = grid.Services().GetServices<TestIgniteServiceSerializable>(SvcName);
+
+            Assert.AreEqual(count, services.Count);
+
+            var svc = services.First();
+
+            Assert.IsNotNull(svc);
+
+            Assert.IsTrue(svc.Initialized);
+
+            Thread.Sleep(100);  // Service runs in a separate thread, wait for it to execute.
+
+            Assert.IsTrue(svc.Executed);
+            Assert.IsFalse(svc.Cancelled);
+
+            Assert.AreEqual(grid.Cluster.LocalNode.Id, svc.NodeId);
+        }
+
+        /// <summary>
+        /// Gets the Ignite configuration.
+        /// </summary>
+        private static IgniteConfiguration Configuration(string springConfigUrl)
+        {
+            return new IgniteConfiguration
+            {
+                SpringConfigUrl = springConfigUrl,
+                JvmClasspath = TestUtils.CreateTestClasspath(),
+                JvmOptions = TestUtils.TestJavaOptions(),
+                PortableConfiguration = new PortableConfiguration
+                {
+                    TypeConfigurations = new List<PortableTypeConfiguration>
+                    {
+                        new PortableTypeConfiguration(typeof(TestIgniteServicePortable)),
+                        new PortableTypeConfiguration(typeof(TestIgniteServicePortableErr)),
+                        new PortableTypeConfiguration(typeof(PortableObject))
+                    }
+                }
+            };
+        }
+
+        /// <summary>
+        /// Gets the services.
+        /// </summary>
+        protected virtual IServices Services
+        {
+            get { return Grid1.Services(); }
+        }
+
+        /// <summary>
+        /// Test service interface for proxying.
+        /// </summary>
+        private interface ITestIgniteService
+        {
+            int TestProperty { get; set; }
+
+            /** */
+            bool Initialized { get; }
+
+            /** */
+            bool Cancelled { get; }
+
+            /** */
+            bool Executed { get; }
+
+            /** */
+            Guid NodeId { get; }
+
+            /** */
+            string LastCallContextName { get; }
+
+            /** */
+            object Method(object arg);
+
+            /** */
+            object ErrMethod(object arg);
+        }
+
+        /// <summary>
+        /// Test service interface for proxy usage.
+        /// Has some of the original interface members with different signatures.
+        /// </summary>
+        private interface ITestIgniteServiceProxyInterface
+        {
+            /** */
+            Guid NodeId { get; }
+
+            /** */
+            object TestProperty { get; set; }
+
+            /** */
+            int Method(int arg);
+        }
+
+        #pragma warning disable 649
+
+        /// <summary>
+        /// Test serializable service.
+        /// </summary>
+        [Serializable]
+        private class TestIgniteServiceSerializable : IService, ITestIgniteService
+        {
+            /** */
+            [InstanceResource]
+            private IIgnite _grid;
+
+            /** <inheritdoc /> */
+            public int TestProperty { get; set; }
+
+            /** <inheritdoc /> */
+            public bool Initialized { get; private set; }
+
+            /** <inheritdoc /> */
+            public bool Cancelled { get; private set; }
+
+            /** <inheritdoc /> */
+            public bool Executed { get; private set; }
+
+            /** <inheritdoc /> */
+            public Guid NodeId
+            {
+                get { return _grid.Cluster.LocalNode.Id; }
+            }
+
+            /** <inheritdoc /> */
+            public string LastCallContextName { get; private set; }
+
+            /** */
+            public bool ThrowInit { get; set; }
+
+            /** */
+            public bool ThrowExecute { get; set; }
+
+            /** */
+            public bool ThrowCancel { get; set; }
+
+            /** */
+            public object Method(object arg)
+            {
+                return arg;
+            }
+
+            /** */
+            public object ErrMethod(object arg)
+            {
+                throw new ArgumentNullException("arg", "ExpectedException");
+            }
+
+            /** <inheritdoc /> */
+            public void Init(IServiceContext context)
+            {
+                if (ThrowInit) 
+                    throw new Exception("Expected exception");
+
+                CheckContext(context);
+
+                Assert.IsFalse(context.IsCancelled);
+                Initialized = true;
+            }
+
+            /** <inheritdoc /> */
+            public void Execute(IServiceContext context)
+            {
+                if (ThrowExecute)
+                    throw new Exception("Expected exception");
+
+                CheckContext(context);
+
+                Assert.IsFalse(context.IsCancelled);
+                Assert.IsTrue(Initialized);
+                Assert.IsFalse(Cancelled);
+
+                Executed = true;
+            }
+
+            /** <inheritdoc /> */
+            public void Cancel(IServiceContext context)
+            {
+                if (ThrowCancel)
+                    throw new Exception("Expected exception");
+
+                CheckContext(context);
+
+                Assert.IsTrue(context.IsCancelled);
+
+                Cancelled = true;
+            }
+
+            /// <summary>
+            /// Checks the service context.
+            /// </summary>
+            private void CheckContext(IServiceContext context)
+            {
+                LastCallContextName = context.Name;
+
+                if (context.AffinityKey != null && !(context.AffinityKey is int))
+                {
+                    var portableObject = context.AffinityKey as IPortableObject;
+                    
+                    var key = portableObject != null
+                        ? portableObject.Deserialize<PortableObject>()
+                        : (PortableObject) context.AffinityKey;
+
+                    Assert.AreEqual(AffKey, key.Val);
+                }
+
+                Assert.IsNotNull(_grid);
+
+                Assert.IsTrue(context.Name.StartsWith(SvcName));
+                Assert.AreNotEqual(Guid.Empty, context.ExecutionId);
+            }
+        }
+
+        /// <summary>
+        /// Test portable service.
+        /// </summary>
+        private class TestIgniteServicePortable : TestIgniteServiceSerializable, IPortableMarshalAware
+        {
+            /** <inheritdoc /> */
+            public void WritePortable(IPortableWriter writer)
+            {
+                writer.WriteInt("TestProp", TestProperty);
+            }
+
+            /** <inheritdoc /> */
+            public void ReadPortable(IPortableReader reader)
+            {
+                TestProperty = reader.ReadInt("TestProp");
+            }
+        }
+
+        /// <summary>
+        /// Test portable service with exceptions in marshalling.
+        /// </summary>
+        private class TestIgniteServicePortableErr : TestIgniteServiceSerializable, IPortableMarshalAware
+        {
+            /** */
+            public bool ThrowOnWrite { get; set; }
+
+            /** <inheritdoc /> */
+            public void WritePortable(IPortableWriter writer)
+            {
+                writer.WriteInt("TestProp", TestProperty);
+                
+                if (ThrowOnWrite)
+                    throw new Exception("Expected exception");
+            }
+
+            /** <inheritdoc /> */
+            public void ReadPortable(IPortableReader reader)
+            {
+                TestProperty = reader.ReadInt("TestProp");
+                
+                throw new Exception("Expected exception");
+            }
+        }
+
+        /// <summary>
+        /// Test node filter.
+        /// </summary>
+        [Serializable]
+        private class NodeFilter : IClusterNodeFilter
+        {
+            /// <summary>
+            /// Gets or sets the node identifier.
+            /// </summary>
+            public Guid NodeId { get; set; }
+
+            /** <inheritdoc /> */
+            public bool Invoke(IClusterNode node)
+            {
+                return node.Id == NodeId;
+            }
+        }
+
+        /// <summary>
+        /// Portable object.
+        /// </summary>
+        private class PortableObject
+        {
+            public int Val { get; set; }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTestAsync.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTestAsync.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTestAsync.cs
new file mode 100644
index 0000000..68ae93e
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTestAsync.cs
@@ -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.
+ */
+
+namespace Apache.Ignite.Core.Tests.Services
+{
+    using Apache.Ignite.Core.Services;
+
+    /// <summary>
+    /// Services async tests.
+    /// </summary>
+    public class ServicesTestAsync : ServicesTest
+    {
+        /** <inheritdoc /> */
+        protected override IServices Services
+        {
+            get { return new ServicesAsyncWrapper(Grid1.Services()); }
+        }
+    }
+}
\ No newline at end of file


[41/45] ignite git commit: Platforms version update (cherry picked from commit c631a37)

Posted by ak...@apache.org.
Platforms version update
(cherry picked from commit c631a37)


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

Branch: refs/heads/ignite-843
Commit: f0939605882fd2d696caeba54983ac702f818ece
Parents: b1a9771
Author: Anton Vinogradov <av...@apache.org>
Authored: Fri Sep 4 17:28:13 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Fri Sep 4 17:29:14 2015 +0300

----------------------------------------------------------------------
 pom.xml | 118 +++++++++++++++++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 118 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f0939605/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index a380d13..b506889 100644
--- a/pom.xml
+++ b/pom.xml
@@ -737,6 +737,124 @@
                 </plugins>
             </build>
         </profile>
+
+        <profile>
+            <id>update-versions</id>
+            <!-- updates dotnet & cpp versions -->
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-antrun-plugin</artifactId>
+                        <version>1.7</version>
+                        <executions>
+                            <execution>
+                                <id>update-versions</id>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                                <phase>validate</phase>
+                                <configuration>
+                                    <target>
+                                        <script language="javascript">
+                                            function setClientVersion(ggVer, clientVer) {
+                                            var p = project.getProperty(ggVer);
+
+                                            if (java.util.regex.Pattern.matches(".*-p\\d+", p))
+                                            project.setProperty(clientVer, p.replace("-p", "."));
+                                            else
+                                            if (java.util.regex.Pattern.matches(".*-[a-zA-Z]+\\d+.*", p))
+                                            project.setProperty(clientVer, p.replaceAll("-[a-zA-Z]+(\\d+)?.*", ".$1"));
+                                            else
+                                            project.setProperty(clientVer, p);
+                                            }
+
+                                            function fix(dest, source) {
+                                            project.setProperty(dest, project.getProperty(source).replace("-SNAPSHOT",
+                                            ""));
+                                            }
+
+                                            fix('ignite.version.fixed', 'project.version');
+                                            fix('new.ignite.version.fixed', 'new.ignite.version');
+                                            setClientVersion('ignite.version.fixed', 'old.client.version');
+                                            setClientVersion('new.ignite.version.fixed', 'new.client.version');
+                                        </script>
+
+                                        <echo message="Update ignite.version in dotnet client"/>
+                                        <echo message="${new.client.version}"/>
+
+                                        <replaceregexp byline="true" encoding="UTF-8">
+                                            <regexp
+                                                pattern="(\[assembly:\s*Assembly\w*Version\w*\(&quot;)\d.\d.\d(.\d)?(&quot;\)\])"/>
+                                            <substitution expression="\1${new.client.version}\3"/>
+                                            <fileset dir="${basedir}/">
+                                                <include name="**/AssemblyInfo.cs"/>
+                                                <include name="**/AssemblyInfo.cpp"/>
+                                            </fileset>
+                                        </replaceregexp>
+
+                                        <echo message="Update ignite.version in cpp client"/>
+                                        <replaceregexp byline="true" encoding="UTF-8">
+                                            <regexp pattern="(AC_INIT.+\[)\d.\d.\d(.\d)?(\].+)"/>
+                                            <substitution expression="\1${new.client.version}\3"/>
+                                            <fileset dir="${basedir}/">
+                                                <include name="**/configure.ac"/>
+                                            </fileset>
+                                        </replaceregexp>
+
+                                        <replaceregexp byline="true" encoding="UTF-8">
+                                            <regexp pattern="(define GG_VERSION_STR_WIN &quot;)\d.\d.\d(.\d)?(&quot;)"/>
+                                            <substitution expression="\1${new.client.version}\3"/>
+                                            <fileset dir="${basedir}/">
+                                                <include name="**/resource.h"/>
+                                            </fileset>
+                                        </replaceregexp>
+
+                                        <replaceregexp byline="true" encoding="UTF-16">
+                                            <regexp pattern="(Version&quot;, &quot;)\d.\d.\d.\d(&quot;)"/>
+                                            <substitution expression="\1${new.client.version}\2"/>
+                                            <fileset dir="${basedir}/">
+                                                <include name="**/Resource.rc"/>
+                                            </fileset>
+                                        </replaceregexp>
+
+                                        <script language="javascript">
+                                            function setBinVersion(clientVer, binVer) {
+                                            var p = project.getProperty(clientVer).replace(".", ",");
+
+                                            if (p.split(',').length == 3)
+                                            project.setProperty(binVer, p + ',0');
+                                            else
+                                            project.setProperty(binVer, p);
+                                            }
+
+                                            setBinVersion('old.client.version', 'old.bin.version');
+                                            setBinVersion('new.client.version', 'new.bin.version');
+                                        </script>
+
+                                        <replaceregexp byline="true" encoding="UTF-8">
+                                            <regexp pattern="(define GG_VERSION_BIN_WIN )\d,\d,\d,\d"/>
+                                            <substitution expression="\1${new.bin.version}"/>
+                                            <fileset dir="${basedir}/">
+                                                <include name="**/resource.h"/>
+                                            </fileset>
+                                        </replaceregexp>
+
+                                        <replaceregexp byline="true" encoding="UTF-16">
+                                            <regexp pattern="(VERSION )\d,\d,\d,\d"/>
+                                            <substitution expression="\1${new.bin.version}"/>
+                                            <fileset dir="${basedir}/">
+                                                <include name="**/Resource.rc"/>
+                                            </fileset>
+                                        </replaceregexp>
+                                    </target>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
     </profiles>
 
     <build>


[25/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Io/PortableAbstractStream.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Io/PortableAbstractStream.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Io/PortableAbstractStream.cs
new file mode 100644
index 0000000..38a19ab
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Io/PortableAbstractStream.cs
@@ -0,0 +1,1299 @@
+/*
+ * 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.Portable.IO
+{
+    using System;
+    using System.Diagnostics.CodeAnalysis;
+    using System.IO;
+    using System.Reflection;
+    using System.Text;
+
+    /// <summary>
+    /// Base class for managed and unmanaged data streams.
+    /// </summary>
+    [CLSCompliant(false)]
+    internal unsafe abstract class PortableAbstractStream : IPortableStream
+    {
+        /// <summary>
+        /// Array copy delegate.
+        /// </summary>
+        delegate void MemCopy(byte* a1, byte* a2, int len);
+
+        /** memcpy function handle. */
+        private static readonly MemCopy Memcpy;
+
+        /** Whether src and dest arguments are inverted. */
+        private static readonly bool MemcpyInverted;
+
+        /** Byte: zero. */
+        protected const byte ByteZero = 0;
+
+        /** Byte: one. */
+        protected const byte ByteOne = 1;
+
+        /** LITTLE_ENDIAN flag. */
+        protected static readonly bool LittleEndian = BitConverter.IsLittleEndian;
+
+        /** Position. */
+        protected int Pos;
+
+        /** Disposed flag. */
+        private bool _disposed;
+
+        /// <summary>
+        /// Static initializer.
+        /// </summary>
+        [SuppressMessage("Microsoft.Design", "CA1065:DoNotRaiseExceptionsInUnexpectedLocations")]
+        static PortableAbstractStream()
+        {
+            Type type = typeof(Buffer);
+
+            const BindingFlags flags = BindingFlags.Static | BindingFlags.NonPublic;
+            Type[] paramTypes = { typeof(byte*), typeof(byte*), typeof(int) };
+
+            // Assume .Net 4.5.
+            MethodInfo mthd = type.GetMethod("Memcpy", flags, null, paramTypes, null);
+
+            MemcpyInverted = true;
+
+            if (mthd == null)
+            {
+                // Assume .Net 4.0.
+                mthd = type.GetMethod("memcpyimpl", flags, null, paramTypes, null);
+
+                MemcpyInverted = false;
+
+                if (mthd == null)
+                    throw new InvalidOperationException("Unable to get memory copy function delegate.");
+            }
+
+            Memcpy = (MemCopy)Delegate.CreateDelegate(typeof(MemCopy), mthd);
+        }
+
+        /// <summary>
+        /// Write byte.
+        /// </summary>
+        /// <param name="val">Byte value.</param>
+        public abstract void WriteByte(byte val);
+
+        /// <summary>
+        /// Read byte.
+        /// </summary>
+        /// <returns>
+        /// Byte value.
+        /// </returns>
+        public abstract byte ReadByte();
+
+        /// <summary>
+        /// Write byte array.
+        /// </summary>
+        /// <param name="val">Byte array.</param>
+        public abstract void WriteByteArray(byte[] val);
+
+        /// <summary>
+        /// Internal routine to write byte array.
+        /// </summary>
+        /// <param name="val">Byte array.</param>
+        /// <param name="data">Data pointer.</param>
+        protected void WriteByteArray0(byte[] val, byte* data)
+        {
+            fixed (byte* val0 = val)
+            {
+                CopyMemory(val0, data, val.Length);
+            }
+        }
+
+        /// <summary>
+        /// Read byte array.
+        /// </summary>
+        /// <param name="cnt">Count.</param>
+        /// <returns>
+        /// Byte array.
+        /// </returns>
+        public abstract byte[] ReadByteArray(int cnt);
+
+        /// <summary>
+        /// Internal routine to read byte array.
+        /// </summary>
+        /// <param name="len">Array length.</param>
+        /// <param name="data">Data pointer.</param>
+        /// <returns>Byte array</returns>
+        protected byte[] ReadByteArray0(int len, byte* data)
+        {
+            byte[] res = new byte[len];
+
+            fixed (byte* res0 = res)
+            {
+                CopyMemory(data, res0, len);
+            }
+
+            return res;
+        }
+
+        /// <summary>
+        /// Write bool.
+        /// </summary>
+        /// <param name="val">Bool value.</param>
+        public void WriteBool(bool val)
+        {
+            WriteByte(val ? ByteOne : ByteZero);
+        }
+
+        /// <summary>
+        /// Read bool.
+        /// </summary>
+        /// <returns>
+        /// Bool value.
+        /// </returns>
+        public bool ReadBool()
+        {
+            return ReadByte() == ByteOne;
+        }
+
+        /// <summary>
+        /// Write bool array.
+        /// </summary>
+        /// <param name="val">Bool array.</param>
+        public abstract void WriteBoolArray(bool[] val);
+
+        /// <summary>
+        /// Internal routine to write bool array.
+        /// </summary>
+        /// <param name="val">Bool array.</param>
+        /// <param name="data">Data pointer.</param>
+        protected void WriteBoolArray0(bool[] val, byte* data)
+        {
+            fixed (bool* val0 = val)
+            {
+                CopyMemory((byte*)val0, data, val.Length);
+            }
+        }
+
+        /// <summary>
+        /// Read bool array.
+        /// </summary>
+        /// <param name="cnt">Count.</param>
+        /// <returns>
+        /// Bool array.
+        /// </returns>
+        public abstract bool[] ReadBoolArray(int cnt);
+
+        /// <summary>
+        /// Internal routine to read bool array.
+        /// </summary>
+        /// <param name="len">Array length.</param>
+        /// <param name="data">Data pointer.</param>
+        /// <returns>Bool array</returns>
+        protected bool[] ReadBoolArray0(int len, byte* data)
+        {
+            bool[] res = new bool[len];
+
+            fixed (bool* res0 = res)
+            {
+                CopyMemory(data, (byte*)res0, len);
+            }
+
+            return res;
+        }
+
+        /// <summary>
+        /// Write short.
+        /// </summary>
+        /// <param name="val">Short value.</param>
+        public abstract void WriteShort(short val);
+
+        /// <summary>
+        /// Internal routine to write short value.
+        /// </summary>
+        /// <param name="val">Short value.</param>
+        /// <param name="data">Data pointer.</param>
+        protected void WriteShort0(short val, byte* data)
+        {
+            if (LittleEndian)
+                *((short*)data) = val;
+            else
+            {
+                byte* valPtr = (byte*)&val;
+
+                data[0] = valPtr[1];
+                data[1] = valPtr[0];
+            }
+        }
+
+        /// <summary>
+        /// Read short.
+        /// </summary>
+        /// <returns>
+        /// Short value.
+        /// </returns>
+        public abstract short ReadShort();
+
+        /// <summary>
+        /// Internal routine to read short value.
+        /// </summary>
+        /// <param name="data">Data pointer.</param>
+        /// <returns>Short value</returns>
+        protected short ReadShort0(byte* data)
+        {
+            short val;
+
+            if (LittleEndian)
+                val = *((short*)data);
+            else
+            {
+                byte* valPtr = (byte*)&val;
+
+                valPtr[0] = data[1];
+                valPtr[1] = data[0];
+            }
+
+            return val;
+        }
+
+        /// <summary>
+        /// Write short array.
+        /// </summary>
+        /// <param name="val">Short array.</param>
+        public abstract void WriteShortArray(short[] val);
+
+        /// <summary>
+        /// Internal routine to write short array.
+        /// </summary>
+        /// <param name="val">Short array.</param>
+        /// <param name="data">Data pointer.</param>
+        /// <param name="cnt">Bytes count.</param>
+        protected void WriteShortArray0(short[] val, byte* data, int cnt)
+        {
+            if (LittleEndian)
+            {
+                fixed (short* val0 = val)
+                {
+                    CopyMemory((byte*)val0, data, cnt);
+                }
+            }
+            else
+            {
+                byte* curPos = data;
+
+                for (int i = 0; i < val.Length; i++)
+                {
+                    short val0 = val[i];
+
+                    byte* valPtr = (byte*)&(val0);
+                    
+                    *curPos++ = valPtr[1];
+                    *curPos++ = valPtr[0];
+                }
+            }
+        }
+
+        /// <summary>
+        /// Read short array.
+        /// </summary>
+        /// <param name="cnt">Count.</param>
+        /// <returns>
+        /// Short array.
+        /// </returns>
+        public abstract short[] ReadShortArray(int cnt);
+
+        /// <summary>
+        /// Internal routine to read short array.
+        /// </summary>
+        /// <param name="len">Array length.</param>
+        /// <param name="data">Data pointer.</param>
+        /// <param name="cnt">Bytes count.</param>
+        /// <returns>Short array</returns>
+        protected short[] ReadShortArray0(int len, byte* data, int cnt)
+        {
+            short[] res = new short[len];
+
+            if (LittleEndian)
+            {
+                fixed (short* res0 = res)
+                {
+                    CopyMemory(data, (byte*)res0, cnt);
+                }
+            }
+            else
+            {
+                for (int i = 0; i < len; i++)
+                {
+                    short val;
+
+                    byte* valPtr = (byte*)&val;
+
+                    valPtr[1] = *data++;
+                    valPtr[0] = *data++;
+
+                    res[i] = val;
+                }
+            }
+
+            return res;
+        }
+
+        /// <summary>
+        /// Write char.
+        /// </summary>
+        /// <param name="val">Char value.</param>
+        public void WriteChar(char val)
+        {
+            WriteShort(*(short*)(&val));
+        }
+
+        /// <summary>
+        /// Read char.
+        /// </summary>
+        /// <returns>
+        /// Char value.
+        /// </returns>
+        public char ReadChar()
+        {
+            short val = ReadShort();
+
+            return *(char*)(&val);
+        }
+
+        /// <summary>
+        /// Write char array.
+        /// </summary>
+        /// <param name="val">Char array.</param>
+        public abstract void WriteCharArray(char[] val);
+
+        /// <summary>
+        /// Internal routine to write char array.
+        /// </summary>
+        /// <param name="val">Char array.</param>
+        /// <param name="data">Data pointer.</param>
+        /// <param name="cnt">Bytes count.</param>
+        protected void WriteCharArray0(char[] val, byte* data, int cnt)
+        {
+            if (LittleEndian)
+            {
+                fixed (char* val0 = val)
+                {
+                    CopyMemory((byte*)val0, data, cnt);
+                }
+            }
+            else
+            {
+                byte* curPos = data;
+
+                for (int i = 0; i < val.Length; i++)
+                {
+                    char val0 = val[i];
+
+                    byte* valPtr = (byte*)&(val0);
+
+                    *curPos++ = valPtr[1];
+                    *curPos++ = valPtr[0];
+                }
+            }
+        }
+
+        /// <summary>
+        /// Read char array.
+        /// </summary>
+        /// <param name="cnt">Count.</param>
+        /// <returns>
+        /// Char array.
+        /// </returns>
+        public abstract char[] ReadCharArray(int cnt);
+
+        /// <summary>
+        /// Internal routine to read char array.
+        /// </summary>
+        /// <param name="len">Count.</param>
+        /// <param name="data">Data pointer.</param>
+        /// <param name="cnt">Bytes count.</param>
+        /// <returns>Char array</returns>
+        protected char[] ReadCharArray0(int len, byte* data, int cnt)
+        {
+            char[] res = new char[len];
+
+            if (LittleEndian)
+            {
+                fixed (char* res0 = res)
+                {
+                    CopyMemory(data, (byte*)res0, cnt);
+                }
+            }
+            else
+            {
+                for (int i = 0; i < len; i++)
+                {
+                    char val;
+
+                    byte* valPtr = (byte*)&val;
+
+                    valPtr[1] = *data++;
+                    valPtr[0] = *data++;
+
+                    res[i] = val;
+                }
+            }
+
+            return res;
+        }
+
+        /// <summary>
+        /// Write int.
+        /// </summary>
+        /// <param name="val">Int value.</param>
+        public abstract void WriteInt(int val);
+
+        /// <summary>
+        /// Write int to specific position.
+        /// </summary>
+        /// <param name="writePos">Position.</param>
+        /// <param name="val">Value.</param>
+        public abstract void WriteInt(int writePos, int val);
+
+        /// <summary>
+        /// Internal routine to write int value.
+        /// </summary>
+        /// <param name="val">Int value.</param>
+        /// <param name="data">Data pointer.</param>
+        protected void WriteInt0(int val, byte* data)
+        {
+            if (LittleEndian)
+                *((int*)data) = val;
+            else
+            {
+                byte* valPtr = (byte*)&val;
+
+                data[0] = valPtr[3];
+                data[1] = valPtr[2];
+                data[2] = valPtr[1];
+                data[3] = valPtr[0];
+            }
+        }
+
+        /// <summary>
+        /// Read int.
+        /// </summary>
+        /// <returns>
+        /// Int value.
+        /// </returns>
+        public abstract int ReadInt();
+
+        /// <summary>
+        /// Internal routine to read int value.
+        /// </summary>
+        /// <param name="data">Data pointer.</param>
+        /// <returns>Int value</returns>
+        protected int ReadInt0(byte* data) {
+            int val;
+
+            if (LittleEndian)
+                val = *((int*)data);
+            else
+            {
+                byte* valPtr = (byte*)&val;
+
+                valPtr[0] = data[3];
+                valPtr[1] = data[2];
+                valPtr[2] = data[1];
+                valPtr[3] = data[0];
+            }
+            
+            return val;
+        }
+
+        /// <summary>
+        /// Write int array.
+        /// </summary>
+        /// <param name="val">Int array.</param>
+        public abstract void WriteIntArray(int[] val);
+
+        /// <summary>
+        /// Internal routine to write int array.
+        /// </summary>
+        /// <param name="val">Int array.</param>
+        /// <param name="data">Data pointer.</param>
+        /// <param name="cnt">Bytes count.</param>
+        protected void WriteIntArray0(int[] val, byte* data, int cnt)
+        {
+            if (LittleEndian)
+            {
+                fixed (int* val0 = val)
+                {
+                    CopyMemory((byte*)val0, data, cnt);
+                }
+            }
+            else
+            {
+                byte* curPos = data;
+
+                for (int i = 0; i < val.Length; i++)
+                {
+                    int val0 = val[i];
+
+                    byte* valPtr = (byte*)&(val0);
+
+                    *curPos++ = valPtr[3];
+                    *curPos++ = valPtr[2];
+                    *curPos++ = valPtr[1];
+                    *curPos++ = valPtr[0];
+                }
+            }
+        }
+
+        /// <summary>
+        /// Read int array.
+        /// </summary>
+        /// <param name="cnt">Count.</param>
+        /// <returns>
+        /// Int array.
+        /// </returns>
+        public abstract int[] ReadIntArray(int cnt);
+
+        /// <summary>
+        /// Internal routine to read int array.
+        /// </summary>
+        /// <param name="len">Count.</param>
+        /// <param name="data">Data pointer.</param>
+        /// <param name="cnt">Bytes count.</param>
+        /// <returns>Int array</returns>
+        protected int[] ReadIntArray0(int len, byte* data, int cnt)
+        {
+            int[] res = new int[len];
+
+            if (LittleEndian)
+            {
+                fixed (int* res0 = res)
+                {
+                    CopyMemory(data, (byte*)res0, cnt);
+                }
+            }
+            else
+            {
+                for (int i = 0; i < len; i++)
+                {
+                    int val;
+
+                    byte* valPtr = (byte*)&val;
+
+                    valPtr[3] = *data++;
+                    valPtr[2] = *data++;
+                    valPtr[1] = *data++;
+                    valPtr[0] = *data++;
+
+                    res[i] = val;
+                }
+            }
+
+            return res;
+        }
+
+        /// <summary>
+        /// Write float.
+        /// </summary>
+        /// <param name="val">Float value.</param>
+        public void WriteFloat(float val)
+        {
+            int val0 = *(int*)(&val);
+
+            WriteInt(val0);
+        }
+
+        /// <summary>
+        /// Read float.
+        /// </summary>
+        /// <returns>
+        /// Float value.
+        /// </returns>
+        public float ReadFloat()
+        {
+            int val = ReadInt();
+
+            return *(float*)(&val);
+        }
+
+        /// <summary>
+        /// Write float array.
+        /// </summary>
+        /// <param name="val">Float array.</param>
+        public abstract void WriteFloatArray(float[] val);
+
+        /// <summary>
+        /// Internal routine to write float array.
+        /// </summary>
+        /// <param name="val">Int array.</param>
+        /// <param name="data">Data pointer.</param>
+        /// <param name="cnt">Bytes count.</param>
+        protected void WriteFloatArray0(float[] val, byte* data, int cnt)
+        {
+            if (LittleEndian)
+            {
+                fixed (float* val0 = val)
+                {
+                    CopyMemory((byte*)val0, data, cnt);
+                }
+            }
+            else
+            {
+                byte* curPos = data;
+
+                for (int i = 0; i < val.Length; i++)
+                {
+                    float val0 = val[i];
+
+                    byte* valPtr = (byte*)&(val0);
+
+                    *curPos++ = valPtr[3];
+                    *curPos++ = valPtr[2];
+                    *curPos++ = valPtr[1];
+                    *curPos++ = valPtr[0];
+                }
+            }
+        }
+
+        /// <summary>
+        /// Read float array.
+        /// </summary>
+        /// <param name="cnt">Count.</param>
+        /// <returns>
+        /// Float array.
+        /// </returns>
+        public abstract float[] ReadFloatArray(int cnt);
+
+        /// <summary>
+        /// Internal routine to read float array.
+        /// </summary>
+        /// <param name="len">Count.</param>
+        /// <param name="data">Data pointer.</param>
+        /// <param name="cnt">Bytes count.</param>
+        /// <returns>Float array</returns>
+        protected float[] ReadFloatArray0(int len, byte* data, int cnt)
+        {
+            float[] res = new float[len];
+
+            if (LittleEndian)
+            {
+                fixed (float* res0 = res)
+                {
+                    CopyMemory(data, (byte*)res0, cnt);
+                }
+            }
+            else
+            {
+                for (int i = 0; i < len; i++)
+                {
+                    int val;
+
+                    byte* valPtr = (byte*)&val;
+
+                    valPtr[3] = *data++;
+                    valPtr[2] = *data++;
+                    valPtr[1] = *data++;
+                    valPtr[0] = *data++;
+
+                    res[i] = val;
+                }
+            }
+
+            return res;
+        }
+
+        /// <summary>
+        /// Write long.
+        /// </summary>
+        /// <param name="val">Long value.</param>
+        public abstract void WriteLong(long val);
+
+        /// <summary>
+        /// Internal routine to write long value.
+        /// </summary>
+        /// <param name="val">Long value.</param>
+        /// <param name="data">Data pointer.</param>
+        protected void WriteLong0(long val, byte* data)
+        {
+            if (LittleEndian)
+                *((long*)data) = val;
+            else
+            {
+                byte* valPtr = (byte*)&val;
+
+                data[0] = valPtr[7];
+                data[1] = valPtr[6];
+                data[2] = valPtr[5];
+                data[3] = valPtr[4];
+                data[4] = valPtr[3];
+                data[5] = valPtr[2];
+                data[6] = valPtr[1];
+                data[7] = valPtr[0];
+            }
+        }
+
+        /// <summary>
+        /// Read long.
+        /// </summary>
+        /// <returns>
+        /// Long value.
+        /// </returns>
+        public abstract long ReadLong();
+
+        /// <summary>
+        /// Internal routine to read long value.
+        /// </summary>
+        /// <param name="data">Data pointer.</param>
+        /// <returns>Long value</returns>
+        protected long ReadLong0(byte* data)
+        {
+            long val;
+
+            if (LittleEndian)
+                val = *((long*)data);
+            else
+            {
+                byte* valPtr = (byte*)&val;
+
+                valPtr[0] = data[7];
+                valPtr[1] = data[6];
+                valPtr[2] = data[5];
+                valPtr[3] = data[4];
+                valPtr[4] = data[3];
+                valPtr[5] = data[2];
+                valPtr[6] = data[1];
+                valPtr[7] = data[0];
+            }
+
+            return val;
+        }
+
+        /// <summary>
+        /// Write long array.
+        /// </summary>
+        /// <param name="val">Long array.</param>
+        public abstract void WriteLongArray(long[] val);
+
+        /// <summary>
+        /// Internal routine to write long array.
+        /// </summary>
+        /// <param name="val">Long array.</param>
+        /// <param name="data">Data pointer.</param>
+        /// <param name="cnt">Bytes count.</param>
+        protected void WriteLongArray0(long[] val, byte* data, int cnt)
+        {
+            if (LittleEndian)
+            {
+                fixed (long* val0 = val)
+                {
+                    CopyMemory((byte*)val0, data, cnt);
+                }
+            }
+            else
+            {
+                byte* curPos = data;
+
+                for (int i = 0; i < val.Length; i++)
+                {
+                    long val0 = val[i];
+
+                    byte* valPtr = (byte*)&(val0);
+
+                    *curPos++ = valPtr[7];
+                    *curPos++ = valPtr[6];
+                    *curPos++ = valPtr[5];
+                    *curPos++ = valPtr[4];
+                    *curPos++ = valPtr[3];
+                    *curPos++ = valPtr[2];
+                    *curPos++ = valPtr[1];
+                    *curPos++ = valPtr[0];
+                }
+            }
+        }
+
+        /// <summary>
+        /// Read long array.
+        /// </summary>
+        /// <param name="cnt">Count.</param>
+        /// <returns>
+        /// Long array.
+        /// </returns>
+        public abstract long[] ReadLongArray(int cnt);
+
+        /// <summary>
+        /// Internal routine to read long array.
+        /// </summary>
+        /// <param name="len">Count.</param>
+        /// <param name="data">Data pointer.</param>
+        /// <param name="cnt">Bytes count.</param>
+        /// <returns>Long array</returns>
+        protected long[] ReadLongArray0(int len, byte* data, int cnt)
+        {
+            long[] res = new long[len];
+
+            if (LittleEndian)
+            {
+                fixed (long* res0 = res)
+                {
+                    CopyMemory(data, (byte*)res0, cnt);
+                }
+            }
+            else
+            {
+                for (int i = 0; i < len; i++)
+                {
+                    long val;
+
+                    byte* valPtr = (byte*)&val;
+
+                    valPtr[7] = *data++;
+                    valPtr[6] = *data++;
+                    valPtr[5] = *data++;
+                    valPtr[4] = *data++;
+                    valPtr[3] = *data++;
+                    valPtr[2] = *data++;
+                    valPtr[1] = *data++;
+                    valPtr[0] = *data++;
+
+                    res[i] = val;
+                }
+            }
+
+            return res;
+        }
+
+        /// <summary>
+        /// Write double.
+        /// </summary>
+        /// <param name="val">Double value.</param>
+        public void WriteDouble(double val)
+        {
+            long val0 = *(long*)(&val);
+
+            WriteLong(val0);
+        }
+
+        /// <summary>
+        /// Read double.
+        /// </summary>
+        /// <returns>
+        /// Double value.
+        /// </returns>
+        public double ReadDouble()
+        {
+            long val = ReadLong();
+
+            return *(double*)(&val);
+        }
+
+        /// <summary>
+        /// Write double array.
+        /// </summary>
+        /// <param name="val">Double array.</param>
+        public abstract void WriteDoubleArray(double[] val);
+
+        /// <summary>
+        /// Internal routine to write double array.
+        /// </summary>
+        /// <param name="val">Double array.</param>
+        /// <param name="data">Data pointer.</param>
+        /// <param name="cnt">Bytes count.</param>
+        protected void WriteDoubleArray0(double[] val, byte* data, int cnt)
+        {
+            if (LittleEndian)
+            {
+                fixed (double* val0 = val)
+                {
+                    CopyMemory((byte*)val0, data, cnt);
+                }
+            }
+            else
+            {
+                byte* curPos = data;
+
+                for (int i = 0; i < val.Length; i++)
+                {
+                    double val0 = val[i];
+
+                    byte* valPtr = (byte*)&(val0);
+
+                    *curPos++ = valPtr[7];
+                    *curPos++ = valPtr[6];
+                    *curPos++ = valPtr[5];
+                    *curPos++ = valPtr[4];
+                    *curPos++ = valPtr[3];
+                    *curPos++ = valPtr[2];
+                    *curPos++ = valPtr[1];
+                    *curPos++ = valPtr[0];
+                }
+            }
+        }
+
+        /// <summary>
+        /// Read double array.
+        /// </summary>
+        /// <param name="cnt">Count.</param>
+        /// <returns>
+        /// Double array.
+        /// </returns>
+        public abstract double[] ReadDoubleArray(int cnt);
+
+        /// <summary>
+        /// Internal routine to read double array.
+        /// </summary>
+        /// <param name="len">Count.</param>
+        /// <param name="data">Data pointer.</param>
+        /// <param name="cnt">Bytes count.</param>
+        /// <returns>Double array</returns>
+        protected double[] ReadDoubleArray0(int len, byte* data, int cnt)
+        {
+            double[] res = new double[len];
+
+            if (LittleEndian)
+            {
+                fixed (double* res0 = res)
+                {
+                    CopyMemory(data, (byte*)res0, cnt);
+                }
+            }
+            else
+            {
+                for (int i = 0; i < len; i++)
+                {
+                    double val;
+
+                    byte* valPtr = (byte*)&val;
+
+                    valPtr[7] = *data++;
+                    valPtr[6] = *data++;
+                    valPtr[5] = *data++;
+                    valPtr[4] = *data++;
+                    valPtr[3] = *data++;
+                    valPtr[2] = *data++;
+                    valPtr[1] = *data++;
+                    valPtr[0] = *data++;
+
+                    res[i] = val;
+                }
+            }
+
+            return res;
+        }
+
+        /// <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>
+        public abstract int WriteString(char* chars, int charCnt, int byteCnt, Encoding encoding);
+
+        /// <summary>
+        /// Internal string write routine.
+        /// </summary>
+        /// <param name="chars">Chars.</param>
+        /// <param name="charCnt">Chars count.</param>
+        /// <param name="byteCnt">Bytes count.</param>
+        /// <param name="enc">Encoding.</param>
+        /// <param name="data">Data.</param>
+        /// <returns>Amount of bytes written.</returns>
+        protected int WriteString0(char* chars, int charCnt, int byteCnt, Encoding enc, byte* data)
+        {
+            return enc.GetBytes(chars, charCnt, data, byteCnt);
+        }
+
+        /// <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)
+        {
+            fixed (byte* src0 = src)
+            {
+                Write(src0 + off, cnt);
+            }
+        }
+
+        /// <summary>
+        /// Read arbitrary data.
+        /// </summary>
+        /// <param name="dest">Destination array.</param>
+        /// <param name="off">Offset.</param>
+        /// <param name="cnt">Count.</param>
+        /// <returns>
+        /// Amount of bytes read.
+        /// </returns>
+        public void Read(byte[] dest, int off, int cnt)
+        {
+            fixed (byte* dest0 = dest)
+            {
+                Read(dest0 + off, cnt);
+            }
+        }
+
+        /// <summary>
+        /// Write arbitrary data.
+        /// </summary>
+        /// <param name="src">Source.</param>
+        /// <param name="cnt">Count.</param>
+        public abstract void Write(byte* src, int cnt);
+
+        /// <summary>
+        /// Internal write routine.
+        /// </summary>
+        /// <param name="src">Source.</param>
+        /// <param name="cnt">Count.</param>
+        /// <param name="data">Data (dsetination).</param>
+        protected void WriteInternal(byte* src, int cnt, byte* data)
+        {
+            CopyMemory(src, data + Pos, cnt);
+        }
+
+        /// <summary>
+        /// Read arbitrary data.
+        /// </summary>
+        /// <param name="dest">Destination.</param>
+        /// <param name="cnt">Count.</param>
+        /// <returns></returns>
+        public abstract void Read(byte* dest, int cnt);
+
+        /// <summary>
+        /// Internal read routine.
+        /// </summary>
+        /// <param name="dest">Destination.</param>
+        /// <param name="cnt">Count.</param>
+        /// <param name="data">Data (source).</param>
+        /// <returns>Amount of bytes written.</returns>
+        protected void ReadInternal(byte* dest, int cnt, byte* data)
+        {
+            int cnt0 = Math.Min(Remaining(), cnt);
+
+            CopyMemory(data + Pos, dest, cnt0);
+
+            ShiftRead(cnt0);
+        }
+
+        /// <summary>
+        /// Position.
+        /// </summary>
+        public int Position
+        {
+            get { return Pos; }
+        }
+
+        /// <summary>
+        /// Gets remaining bytes in the stream.
+        /// </summary>
+        /// <returns>
+        /// Remaining bytes.
+        /// </returns>
+        public abstract int Remaining();
+
+        /// <summary>
+        /// Gets underlying array, avoiding copying if possible.
+        /// </summary>
+        /// <returns>
+        /// Underlying array.
+        /// </returns>
+        public abstract byte[] Array();
+
+        /// <summary>
+        /// Gets underlying data in a new array.
+        /// </summary>
+        /// <returns>
+        /// New array with data.
+        /// </returns>
+        public abstract byte[] ArrayCopy();
+
+        /// <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 virtual bool IsSameArray(byte[] arr)
+        {
+            return false;
+        }
+
+        /// <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;
+
+            switch (origin)
+            {
+                case SeekOrigin.Begin:
+                    {
+                        newPos = offset;
+
+                        break;
+                    }
+
+                case SeekOrigin.Current:
+                    {
+                        newPos = Pos + offset;
+
+                        break;
+                    }
+
+                default:
+                    throw new ArgumentException("Unsupported seek origin: " + origin);
+            }
+
+            if (newPos < 0)
+                throw new ArgumentException("Seek before origin: " + newPos);
+
+            EnsureWriteCapacity(newPos);
+
+            Pos = newPos;
+
+            return Pos;
+        }
+
+        /** <inheritdoc /> */
+        public void Dispose()
+        {
+            if (_disposed)
+                return;
+
+            Dispose(true);
+
+            GC.SuppressFinalize(this);
+
+            _disposed = true;
+        }
+
+        /// <summary>
+        /// Performs application-defined tasks associated with freeing, releasing, or resetting unmanaged resources.
+        /// </summary>
+        protected abstract void Dispose(bool disposing);
+
+        /// <summary>
+        /// Ensure capacity for write.
+        /// </summary>
+        /// <param name="cnt">Bytes count.</param>
+        protected abstract void EnsureWriteCapacity(int cnt);
+
+        /// <summary>
+        /// Ensure capacity for write and shift position.
+        /// </summary>
+        /// <param name="cnt">Bytes count.</param>
+        /// <returns>Position before shift.</returns>
+        protected int EnsureWriteCapacityAndShift(int cnt)
+        {
+            int pos0 = Pos;
+
+            EnsureWriteCapacity(Pos + cnt);
+
+            ShiftWrite(cnt);
+
+            return pos0;
+        }
+
+        /// <summary>
+        /// Ensure capacity for read.
+        /// </summary>
+        /// <param name="cnt">Bytes count.</param>
+        protected abstract void EnsureReadCapacity(int cnt);
+
+        /// <summary>
+        /// Ensure capacity for read and shift position.
+        /// </summary>
+        /// <param name="cnt">Bytes count.</param>
+        /// <returns>Position before shift.</returns>
+        protected int EnsureReadCapacityAndShift(int cnt)
+        {
+            int pos0 = Pos;
+
+            EnsureReadCapacity(cnt);
+
+            ShiftRead(cnt);
+
+            return pos0;
+        }
+
+        /// <summary>
+        /// Shift position due to write
+        /// </summary>
+        /// <param name="cnt">Bytes count.</param>
+        protected void ShiftWrite(int cnt)
+        {
+            Pos += cnt;
+        }
+
+        /// <summary>
+        /// Shift position due to read.
+        /// </summary>
+        /// <param name="cnt">Bytes count.</param>
+        protected void ShiftRead(int cnt)
+        {
+            Pos += cnt;
+        }
+
+        /// <summary>
+        /// Calculate new capacity.
+        /// </summary>
+        /// <param name="curCap">Current capacity.</param>
+        /// <param name="reqCap">Required capacity.</param>
+        /// <returns>New capacity.</returns>
+        protected static int Capacity(int curCap, int reqCap)
+        {
+            int newCap;
+
+            if (reqCap < 256)
+                newCap = 256;
+            else
+            {
+                newCap = curCap << 1;
+
+                if (newCap < reqCap)
+                    newCap = reqCap;
+            }
+
+            return newCap;
+        }
+
+        /// <summary>
+        /// Unsafe memory copy routine.
+        /// </summary>
+        /// <param name="src">Source.</param>
+        /// <param name="dest">Destination.</param>
+        /// <param name="len">Length.</param>
+        public static void CopyMemory(byte* src, byte* dest, int len)
+        {
+            if (MemcpyInverted)
+                Memcpy.Invoke(dest, src, len);
+            else
+                Memcpy.Invoke(src, dest, len);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Io/PortableHeapStream.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Io/PortableHeapStream.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Io/PortableHeapStream.cs
new file mode 100644
index 0000000..690f92c
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Io/PortableHeapStream.cs
@@ -0,0 +1,447 @@
+/*
+ * 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.Portable.IO
+{
+    using System;
+    using System.IO;
+    using System.Text;
+
+    /// <summary>
+    /// Portable onheap stream.
+    /// </summary>
+    internal unsafe class PortableHeapStream : PortableAbstractStream
+    {
+        /** Data array. */
+        protected byte[] Data;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="cap">Initial capacity.</param>
+        public PortableHeapStream(int cap)
+        {
+            Data = new byte[cap];
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="data">Data array.</param>
+        public PortableHeapStream(byte[] data)
+        {
+            Data = data;
+        }
+
+        /** <inheritdoc /> */
+        public override void WriteByte(byte val)
+        {
+            int pos0 = EnsureWriteCapacityAndShift(1);
+
+            Data[pos0] = val;
+        }
+
+        /** <inheritdoc /> */
+        public override byte ReadByte()
+        {
+            int pos0 = EnsureReadCapacityAndShift(1);
+
+            return Data[pos0];
+        }
+
+        /** <inheritdoc /> */
+        public override void WriteByteArray(byte[] val)
+        {
+            int pos0 = EnsureWriteCapacityAndShift(val.Length);
+
+            fixed (byte* data0 = Data)
+            {
+                WriteByteArray0(val, data0 + pos0);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public override byte[] ReadByteArray(int cnt)
+        {
+            int pos0 = EnsureReadCapacityAndShift(cnt);
+
+            fixed (byte* data0 = Data)
+            {
+                return ReadByteArray0(cnt, data0 + pos0);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public override void WriteBoolArray(bool[] val)
+        {
+            int pos0 = EnsureWriteCapacityAndShift(val.Length);
+
+            fixed (byte* data0 = Data)
+            {
+                WriteBoolArray0(val, data0 + pos0);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public override bool[] ReadBoolArray(int cnt)
+        {
+            int pos0 = EnsureReadCapacityAndShift(cnt);
+
+            fixed (byte* data0 = Data)
+            {
+                return ReadBoolArray0(cnt, data0 + pos0);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public override void WriteShort(short val)
+        {
+            int pos0 = EnsureWriteCapacityAndShift(2);
+
+            fixed (byte* data0 = Data)
+            {
+                WriteShort0(val, data0 + pos0);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public override short ReadShort()
+        {
+            int pos0 = EnsureReadCapacityAndShift(2);
+
+            fixed (byte* data0 = Data)
+            {
+                return ReadShort0(data0 + pos0);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public override void WriteShortArray(short[] val)
+        {
+            int cnt = val.Length << 1;
+
+            int pos0 = EnsureWriteCapacityAndShift(cnt);
+
+            fixed (byte* data0 = Data)
+            {
+                WriteShortArray0(val, data0 + pos0, cnt);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public override short[] ReadShortArray(int cnt)
+        {
+            int cnt0 = cnt << 1;
+
+            int pos0 = EnsureReadCapacityAndShift(cnt0);
+
+            fixed (byte* data0 = Data)
+            {
+                return ReadShortArray0(cnt, data0 + pos0, cnt0);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public override void WriteCharArray(char[] val)
+        {
+            int cnt = val.Length << 1;
+
+            int pos0 = EnsureWriteCapacityAndShift(cnt);
+
+            fixed (byte* data0 = Data)
+            {
+                WriteCharArray0(val, data0 + pos0, cnt);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public override char[] ReadCharArray(int cnt)
+        {
+            int cnt0 = cnt << 1;
+
+            int pos0 = EnsureReadCapacityAndShift(cnt0);
+
+            fixed (byte* data0 = Data)
+            {
+                return ReadCharArray0(cnt, data0 + pos0, cnt0);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public override void WriteInt(int val)
+        {
+            int pos0 = EnsureWriteCapacityAndShift(4);
+
+            fixed (byte* data0 = Data)
+            {
+                WriteInt0(val, data0 + pos0);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public override void WriteInt(int writePos, int val)
+        {
+            EnsureWriteCapacity(writePos + 4);
+
+            fixed (byte* data0 = Data)
+            {
+                WriteInt0(val, data0 + writePos);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public override int ReadInt()
+        {
+            int pos0 = EnsureReadCapacityAndShift(4);
+
+            fixed (byte* data0 = Data)
+            {
+                return ReadInt0(data0 + pos0);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public override void WriteIntArray(int[] val)
+        {
+            int cnt = val.Length << 2;
+
+            int pos0 = EnsureWriteCapacityAndShift(cnt);
+
+            fixed (byte* data0 = Data)
+            {
+                WriteIntArray0(val, data0 + pos0, cnt);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public override int[] ReadIntArray(int cnt)
+        {
+            int cnt0 = cnt << 2;
+
+            int pos0 = EnsureReadCapacityAndShift(cnt0);
+
+            fixed (byte* data0 = Data)
+            {
+                return ReadIntArray0(cnt, data0 + pos0, cnt0);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public override void WriteFloatArray(float[] val)
+        {
+            int cnt = val.Length << 2;
+
+            int pos0 = EnsureWriteCapacityAndShift(cnt);
+
+            fixed (byte* data0 = Data)
+            {
+                WriteFloatArray0(val, data0 + pos0, cnt);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public override float[] ReadFloatArray(int cnt)
+        {
+            int cnt0 = cnt << 2;
+
+            int pos0 = EnsureReadCapacityAndShift(cnt0);
+
+            fixed (byte* data0 = Data)
+            {
+                return ReadFloatArray0(cnt, data0 + pos0, cnt0);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public override void WriteLong(long val)
+        {
+            int pos0 = EnsureWriteCapacityAndShift(8);
+
+            fixed (byte* data0 = Data)
+            {
+                WriteLong0(val, data0 + pos0);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public override long ReadLong()
+        {
+            int pos0 = EnsureReadCapacityAndShift(8);
+
+            fixed (byte* data0 = Data)
+            {
+                return ReadLong0(data0 + pos0);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public override void WriteLongArray(long[] val)
+        {
+            int cnt = val.Length << 3;
+
+            int pos0 = EnsureWriteCapacityAndShift(cnt);
+
+            fixed (byte* data0 = Data)
+            {
+                WriteLongArray0(val, data0 + pos0, cnt);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public override long[] ReadLongArray(int cnt)
+        {
+            int cnt0 = cnt << 3;
+
+            int pos0 = EnsureReadCapacityAndShift(cnt0);
+
+            fixed (byte* data0 = Data)
+            {
+                return ReadLongArray0(cnt, data0 + pos0, cnt0);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public override void WriteDoubleArray(double[] val)
+        {
+            int cnt = val.Length << 3;
+
+            int pos0 = EnsureWriteCapacityAndShift(cnt);
+
+            fixed (byte* data0 = Data)
+            {
+                WriteDoubleArray0(val, data0 + pos0, cnt);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public override double[] ReadDoubleArray(int cnt)
+        {
+            int cnt0 = cnt << 3;
+
+            int pos0 = EnsureReadCapacityAndShift(cnt0);
+
+            fixed (byte* data0 = Data)
+            {
+                return ReadDoubleArray0(cnt, data0 + pos0, cnt0);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public override int WriteString(char* chars, int charCnt, int byteCnt, Encoding encoding)
+        {
+            int pos0 = EnsureWriteCapacityAndShift(byteCnt);
+
+            int written;
+
+            fixed (byte* data0 = Data)
+            {
+                written = WriteString0(chars, charCnt, byteCnt, encoding, data0 + pos0);
+            }
+
+            return written;
+        }
+
+        /** <inheritdoc /> */
+        public override void Write(byte* src, int cnt)
+        {
+            EnsureWriteCapacity(Pos + cnt);
+
+            fixed (byte* data0 = Data)
+            {
+                WriteInternal(src, cnt, data0);
+            }
+
+            ShiftWrite(cnt);
+        }
+
+        /** <inheritdoc /> */
+        public override void Read(byte* dest, int cnt)
+        {
+            fixed (byte* data0 = Data)
+            {
+                ReadInternal(dest, cnt, data0);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public override int Remaining()
+        {
+            return Data.Length - Pos;
+        }
+
+        /** <inheritdoc /> */
+        public override byte[] Array()
+        {
+            return Data;
+        }
+
+        /** <inheritdoc /> */
+        public override byte[] ArrayCopy()
+        {
+            byte[] copy = new byte[Pos];
+
+            Buffer.BlockCopy(Data, 0, copy, 0, Pos);
+
+            return copy;
+        }
+
+        /** <inheritdoc /> */
+        public override bool IsSameArray(byte[] arr)
+        {
+            return Data == arr;
+        }
+
+        /** <inheritdoc /> */
+        protected override void Dispose(bool disposing)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Internal array.
+        /// </summary>
+        internal byte[] InternalArray
+        {
+            get { return Data; }
+        }
+
+        /** <inheritdoc /> */
+        protected override void EnsureWriteCapacity(int cnt)
+        {
+            if (cnt > Data.Length)
+            {
+                int newCap = Capacity(Data.Length, cnt);
+
+                byte[] data0 = new byte[newCap];
+
+                // Copy the whole initial array length here because it can be changed
+                // from Java without position adjusting.
+                Buffer.BlockCopy(Data, 0, data0, 0, Data.Length);
+
+                Data = data0;
+            }
+        }
+
+        /** <inheritdoc /> */
+        protected override void EnsureReadCapacity(int cnt)
+        {
+            if (Data.Length - Pos < cnt)
+                throw new EndOfStreamException("Not enough data in stream [expected=" + cnt +
+                    ", remaining=" + (Data.Length - Pos) + ']');
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Io/PortableStreamAdapter.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Io/PortableStreamAdapter.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Io/PortableStreamAdapter.cs
new file mode 100644
index 0000000..1d17f89
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Io/PortableStreamAdapter.cs
@@ -0,0 +1,114 @@
+/*
+ * 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.Portable.IO
+{
+    using System;
+    using System.IO;
+
+    /// <summary>
+    /// Adapter providing .Net streaming functionality over the portable stream.
+    /// </summary>
+    internal class PortableStreamAdapter : Stream
+    {
+        /// <summary>
+        /// 
+        /// </summary>
+        private readonly IPortableStream _stream;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="stream">Stream.</param>
+        public PortableStreamAdapter(IPortableStream stream)
+        {
+            _stream = stream;
+        }
+
+        /** <inheritDoc /> */
+        public override void Write(byte[] buffer, int offset, int count)
+        {
+            _stream.Write(buffer, offset, count);
+        }
+
+        /** <inheritDoc /> */
+        public override int Read(byte[] buffer, int offset, int count)
+        {
+            _stream.Read(buffer, offset, count);
+
+            return count;
+        }
+
+        /** <inheritDoc /> */
+        public override void Flush()
+        {
+            // No-op.
+        }
+
+        /** <inheritDoc /> */
+        public override bool CanRead
+        {
+            get { return true; }
+        }
+
+        /** <inheritDoc /> */
+        public override bool CanWrite
+        {
+            get { return true; }
+        }
+
+        /** <inheritDoc /> */
+        public override bool CanSeek
+        {
+            get { return false; }
+        }
+
+        /** <inheritDoc /> */
+        public override long Seek(long offset, SeekOrigin origin)
+        {
+            throw new NotSupportedException("Stream is not seekable.");
+        }
+
+        /** <inheritDoc /> */
+        public override long Position
+        {
+            get
+            {
+                throw new NotSupportedException("Stream is not seekable.");
+            }
+            set
+            {
+                throw new NotSupportedException("Stream is not seekable.");
+            }
+        }
+
+        /** <inheritDoc /> */
+        public override long Length
+        {
+            get 
+            {
+                throw new NotSupportedException("Stream is not seekable.");
+            }
+        }
+
+        /** <inheritDoc /> */
+        public override void SetLength(long value)
+        {
+            throw new NotSupportedException("Stream is not seekable.");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Metadata/IPortableMetadataHandler.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Metadata/IPortableMetadataHandler.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Metadata/IPortableMetadataHandler.cs
new file mode 100644
index 0000000..dc3090f
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Metadata/IPortableMetadataHandler.cs
@@ -0,0 +1,41 @@
+/*
+ * 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.Portable.Metadata
+{
+    using System.Collections.Generic;
+
+    /// <summary>
+    /// Portable metadata handler.
+    /// </summary>
+    public interface IPortableMetadataHandler
+    {
+        /// <summary>
+        /// Callback invoked when named field is written.
+        /// </summary>
+        /// <param name="fieldId">Field ID.</param>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="typeId">Field type ID.</param>
+        void OnFieldWrite(int fieldId, string fieldName, int typeId);
+
+        /// <summary>
+        /// Callback invoked when object write is finished and it is time to collect missing metadata.
+        /// </summary>
+        /// <returns>Collected metadata.</returns>
+        IDictionary<string, int> OnObjectWriteFinished();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Metadata/PortableHashsetMetadataHandler.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Metadata/PortableHashsetMetadataHandler.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Metadata/PortableHashsetMetadataHandler.cs
new file mode 100644
index 0000000..8df5f36
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Metadata/PortableHashsetMetadataHandler.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.Impl.Portable.Metadata
+{
+    using System.Collections.Generic;
+
+    /// <summary>
+    /// Metadata handler which uses hash set to determine whether field was already written or not.
+    /// </summary>
+    internal class PortableHashsetMetadataHandler : IPortableMetadataHandler
+    {
+        /** Empty fields collection. */
+        private static readonly IDictionary<string, int> EmptyFields = new Dictionary<string, int>();
+
+        /** IDs known when serialization starts. */
+        private readonly ICollection<int> _ids;
+
+        /** New fields. */
+        private IDictionary<string, int> _fieldMap;
+
+        /** */
+        private readonly bool _newType;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="ids">IDs.</param>
+        /// <param name="newType">True is metadata for type is not saved.</param>
+        public PortableHashsetMetadataHandler(ICollection<int> ids, bool newType)
+        {
+            _ids = ids;
+            _newType = newType;
+        }
+
+        /** <inheritdoc /> */
+        public void OnFieldWrite(int fieldId, string fieldName, int typeId)
+        {
+            if (!_ids.Contains(fieldId))
+            {
+                if (_fieldMap == null)
+                    _fieldMap = new Dictionary<string, int>();
+
+                if (!_fieldMap.ContainsKey(fieldName))
+                    _fieldMap[fieldName] = typeId;
+            }
+        }
+
+        /** <inheritdoc /> */
+        public IDictionary<string, int> OnObjectWriteFinished()
+        {
+            return _fieldMap ?? (_newType ? EmptyFields : null);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Metadata/PortableMetadataHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Metadata/PortableMetadataHolder.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Metadata/PortableMetadataHolder.cs
new file mode 100644
index 0000000..a3fa90f
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Metadata/PortableMetadataHolder.cs
@@ -0,0 +1,149 @@
+/*
+ * 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.Portable.Metadata
+{
+    using System;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Metadata for particular type.
+    /// </summary>
+    internal class PortableMetadataHolder
+    {
+        /** Type ID. */
+        private readonly int _typeId;
+
+        /** Type name. */
+        private readonly string _typeName;
+
+        /** Affinity key field name. */
+        private readonly string _affKeyFieldName;
+
+        /** Empty metadata when nothig is know about object fields yet. */
+        private readonly IPortableMetadata _emptyMeta;
+
+        /** Collection of know field IDs. */
+        private volatile ICollection<int> _ids;
+
+        /** Last known unmodifiable metadata which is given to the user. */
+        private volatile PortableMetadataImpl _meta;
+
+        /** Saved flag (set if type metadata was saved at least once). */
+        private volatile bool _saved;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="typeId">Type ID.</param>
+        /// <param name="typeName">Type name.</param>
+        /// <param name="affKeyFieldName">Affinity key field name.</param>
+        public PortableMetadataHolder(int typeId, string typeName, string affKeyFieldName)
+        {
+            _typeId = typeId;
+            _typeName = typeName;
+            _affKeyFieldName = affKeyFieldName;
+
+            _emptyMeta = new PortableMetadataImpl(typeId, typeName, null, affKeyFieldName);
+        }
+
+        /// <summary>
+        /// Get saved flag.
+        /// </summary>
+        /// <returns>True if type metadata was saved at least once.</returns>
+        public bool Saved()
+        {
+            return _saved;
+        }
+
+        /// <summary>
+        /// Get current type metadata.
+        /// </summary>
+        /// <returns>Type metadata.</returns>
+        public IPortableMetadata Metadata()
+        {
+            PortableMetadataImpl meta0 = _meta;
+
+            return meta0 != null ? _meta : _emptyMeta;
+        }
+
+        /// <summary>
+        /// Currently cached field IDs.
+        /// </summary>
+        /// <returns>Cached field IDs.</returns>
+        public ICollection<int> FieldIds()
+        {
+            ICollection<int> ids0 = _ids;
+
+            if (_ids == null)
+            {
+                lock (this)
+                {
+                    ids0 = _ids;
+
+                    if (ids0 == null)
+                    {
+                        ids0 = new HashSet<int>();
+
+                        _ids = ids0;
+                    }
+                }
+            }
+
+            return ids0;
+        }
+
+        /// <summary>
+        /// Merge newly sent field metadatas into existing ones.
+        /// </summary>
+        /// <param name="newMap">New field metadatas map.</param>
+        public void Merge(IDictionary<int, Tuple<string, int>> newMap)
+        {
+            _saved = true;
+
+            if (newMap == null || newMap.Count == 0)
+                return;
+
+            lock (this)
+            {
+                // 1. Create copies of the old meta.
+                ICollection<int> ids0 = _ids;
+                PortableMetadataImpl meta0 = _meta;
+
+                ICollection<int> newIds = ids0 != null ? new HashSet<int>(ids0) : new HashSet<int>();
+
+                IDictionary<string, int> newFields = meta0 != null ?
+                    new Dictionary<string, int>(meta0.FieldsMap()) : new Dictionary<string, int>(newMap.Count);
+
+                // 2. Add new fields.
+                foreach (KeyValuePair<int, Tuple<string, int>> newEntry in newMap)
+                {
+                    if (!newIds.Contains(newEntry.Key))
+                        newIds.Add(newEntry.Key);
+
+                    if (!newFields.ContainsKey(newEntry.Value.Item1))
+                        newFields[newEntry.Value.Item1] = newEntry.Value.Item2;
+                }
+
+                // 3. Assign new meta. Order is important here: meta must be assigned before field IDs.
+                _meta = new PortableMetadataImpl(_typeId, _typeName, newFields, _affKeyFieldName);
+                _ids = newIds;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Metadata/PortableMetadataImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Metadata/PortableMetadataImpl.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Metadata/PortableMetadataImpl.cs
new file mode 100644
index 0000000..88b40ad
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Metadata/PortableMetadataImpl.cs
@@ -0,0 +1,200 @@
+/*
+ * 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.Portable.Metadata
+{
+    using System.Collections.Generic;
+    using System.Diagnostics.CodeAnalysis;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Portable metadata implementation.
+    /// </summary>
+    internal class PortableMetadataImpl : IPortableMetadata
+    {
+        /** Empty metadata. */
+        [SuppressMessage("Microsoft.Security", "CA2104:DoNotDeclareReadOnlyMutableReferenceTypes")]
+        public static readonly PortableMetadataImpl EmptyMeta =
+            new PortableMetadataImpl(PortableUtils.TypeObject, PortableTypeNames.TypeNameObject, null, null);
+
+        /** Empty dictionary. */
+        private static readonly IDictionary<string, int> EmptyDict = new Dictionary<string, int>();
+
+        /** Empty list. */
+        private static readonly ICollection<string> EmptyList = new List<string>().AsReadOnly();
+
+        /** Fields. */
+        private readonly IDictionary<string, int> _fields;
+
+        /// <summary>
+        /// Get type name by type ID.
+        /// </summary>
+        /// <param name="typeId">Type ID.</param>
+        /// <returns>Type name.</returns>
+        private static string ConvertTypeName(int typeId)
+        {
+            switch (typeId)
+            {
+                case PortableUtils.TypeBool:
+                    return PortableTypeNames.TypeNameBool;
+                case PortableUtils.TypeByte:
+                    return PortableTypeNames.TypeNameByte;
+                case PortableUtils.TypeShort:
+                    return PortableTypeNames.TypeNameShort;
+                case PortableUtils.TypeChar:
+                    return PortableTypeNames.TypeNameChar;
+                case PortableUtils.TypeInt:
+                    return PortableTypeNames.TypeNameInt;
+                case PortableUtils.TypeLong:
+                    return PortableTypeNames.TypeNameLong;
+                case PortableUtils.TypeFloat:
+                    return PortableTypeNames.TypeNameFloat;
+                case PortableUtils.TypeDouble:
+                    return PortableTypeNames.TypeNameDouble;
+                case PortableUtils.TypeDecimal:
+                    return PortableTypeNames.TypeNameDecimal;
+                case PortableUtils.TypeString:
+                    return PortableTypeNames.TypeNameString;
+                case PortableUtils.TypeGuid:
+                    return PortableTypeNames.TypeNameGuid;
+                case PortableUtils.TypeDate:
+                    return PortableTypeNames.TypeNameDate;
+                case PortableUtils.TypeEnum:
+                    return PortableTypeNames.TypeNameEnum;
+                case PortableUtils.TypePortable:
+                case PortableUtils.TypeObject:
+                    return PortableTypeNames.TypeNameObject;
+                case PortableUtils.TypeArrayBool:
+                    return PortableTypeNames.TypeNameArrayBool;
+                case PortableUtils.TypeArrayByte:
+                    return PortableTypeNames.TypeNameArrayByte;
+                case PortableUtils.TypeArrayShort:
+                    return PortableTypeNames.TypeNameArrayShort;
+                case PortableUtils.TypeArrayChar:
+                    return PortableTypeNames.TypeNameArrayChar;
+                case PortableUtils.TypeArrayInt:
+                    return PortableTypeNames.TypeNameArrayInt;
+                case PortableUtils.TypeArrayLong:
+                    return PortableTypeNames.TypeNameArrayLong;
+                case PortableUtils.TypeArrayFloat:
+                    return PortableTypeNames.TypeNameArrayFloat;
+                case PortableUtils.TypeArrayDouble:
+                    return PortableTypeNames.TypeNameArrayDouble;
+                case PortableUtils.TypeArrayDecimal:
+                    return PortableTypeNames.TypeNameArrayDecimal;
+                case PortableUtils.TypeArrayString:
+                    return PortableTypeNames.TypeNameArrayString;
+                case PortableUtils.TypeArrayGuid:
+                    return PortableTypeNames.TypeNameArrayGuid;
+                case PortableUtils.TypeArrayDate:
+                    return PortableTypeNames.TypeNameArrayDate;
+                case PortableUtils.TypeArrayEnum:
+                    return PortableTypeNames.TypeNameArrayEnum;
+                case PortableUtils.TypeArray:
+                    return PortableTypeNames.TypeNameArrayObject;
+                case PortableUtils.TypeCollection:
+                    return PortableTypeNames.TypeNameCollection;
+                case PortableUtils.TypeDictionary:
+                    return PortableTypeNames.TypeNameMap;
+                default:
+                    throw new PortableException("Invalid type ID: " + typeId);
+            }
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="PortableMetadataImpl" /> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        public PortableMetadataImpl(IPortableRawReader reader)
+        {
+            TypeId = reader.ReadInt();
+            TypeName = reader.ReadString();
+            AffinityKeyFieldName = reader.ReadString();
+            _fields = reader.ReadGenericDictionary<string, int>();
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="typeId">Type ID.</param>
+        /// <param name="typeName">Type name.</param>
+        /// <param name="fields">Fields.</param>
+        /// <param name="affKeyFieldName">Affinity key field name.</param>
+        public PortableMetadataImpl(int typeId, string typeName, IDictionary<string, int> fields,
+            string affKeyFieldName)
+        {
+            TypeId = typeId;
+            TypeName = typeName;
+            AffinityKeyFieldName = affKeyFieldName;
+            _fields = fields;
+        }
+
+        /// <summary>
+        /// Type ID.
+        /// </summary>
+        /// <returns></returns>
+        public int TypeId { get; private set; }
+
+        /// <summary>
+        /// Gets type name.
+        /// </summary>
+        public string TypeName { get; private set; }
+
+        /// <summary>
+        /// Gets field names for that type.
+        /// </summary>
+        public ICollection<string> Fields
+        {
+            get { return _fields != null ? _fields.Keys : EmptyList; }
+        }
+
+        /// <summary>
+        /// Gets field type for the given field name.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>
+        /// Field type.
+        /// </returns>
+        public string FieldTypeName(string fieldName)
+        {
+            if (_fields != null)
+            {
+                int typeId;
+
+                _fields.TryGetValue(fieldName, out typeId);
+
+                return ConvertTypeName(typeId);
+            }
+            
+            return null;
+        }
+
+        /// <summary>
+        /// Gets optional affinity key field name.
+        /// </summary>
+        public string AffinityKeyFieldName { get; private set; }
+
+        /// <summary>
+        /// Gets fields map.
+        /// </summary>
+        /// <returns>Fields map.</returns>
+        public IDictionary<string, int> FieldsMap()
+        {
+            return _fields ?? EmptyDict;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableBuilderField.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableBuilderField.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableBuilderField.cs
new file mode 100644
index 0000000..026d0d4
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/PortableBuilderField.cs
@@ -0,0 +1,73 @@
+/*
+ * 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.Portable
+{
+    using System;
+
+    /// <summary>
+    /// Portable builder field.
+    /// </summary>
+    internal class PortableBuilderField
+    {
+        /** Remove marker object. */
+        public static readonly object RmvMarkerObj = new object();
+
+        /** Remove marker. */
+        public static readonly PortableBuilderField RmvMarker = 
+            new PortableBuilderField(null, RmvMarkerObj);
+
+        /** Type. */
+        private readonly Type _typ;
+
+        /** Value. */
+        private readonly object _val;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="typ">Type.</param>
+        /// <param name="val">Value.</param>
+        public PortableBuilderField(Type typ, object val)
+        {
+            _typ = typ;
+            _val = val;
+        }
+
+        /// <summary>
+        /// Type.
+        /// </summary>
+        public Type Type
+        {
+            get
+            {
+                return _typ;
+            }
+        }
+
+        /// <summary>
+        /// Value.
+        /// </summary>
+        public object Value
+        {
+            get
+            {
+                return _val;
+            }
+        }
+    }
+}


[19/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Resource/ResourceTypeDescriptor.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Resource/ResourceTypeDescriptor.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Resource/ResourceTypeDescriptor.cs
new file mode 100644
index 0000000..de5d4c7
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Resource/ResourceTypeDescriptor.cs
@@ -0,0 +1,291 @@
+/*
+ * 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.Resource
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Reflection;
+    using Apache.Ignite.Core.Cache.Store;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Resource;
+
+    /// <summary>
+    /// Resource type descriptor.
+    /// </summary>
+    internal class ResourceTypeDescriptor
+    {
+        /** Attribute type: InstanceResourceAttribute. */
+        private static readonly Type TypAttrIgnite = typeof(InstanceResourceAttribute);
+
+        /** Attribute type: StoreSessionResourceAttribute. */
+        private static readonly Type TypAttrStoreSes = typeof(StoreSessionResourceAttribute);
+
+        /** Type: IGrid. */
+        private static readonly Type TypIgnite = typeof(IIgnite);
+
+        /** Type: ICacheStoreSession. */
+        private static readonly Type TypStoreSes = typeof (ICacheStoreSession);
+
+        /** Type: ComputeTaskNoResultCacheAttribute. */
+        private static readonly Type TypComputeTaskNoResCache = typeof(ComputeTaskNoResultCacheAttribute);
+
+        /** Cached binding flags. */
+        private static readonly BindingFlags Flags = BindingFlags.Instance | BindingFlags.Public |
+            BindingFlags.NonPublic | BindingFlags.DeclaredOnly;
+
+        /** Ignite injectors. */
+        private readonly IList<IResourceInjector> _igniteInjectors;
+
+        /** Session injectors. */
+        private readonly IList<IResourceInjector> _storeSesInjectors;
+        
+        /** Task "no result cache" flag. */
+        private readonly bool _taskNoResCache;
+        
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="type">Type.</param>
+        internal ResourceTypeDescriptor(Type type)
+        {
+            Collector gridCollector = new Collector(TypAttrIgnite, TypIgnite);
+            Collector storeSesCollector = new Collector(TypAttrStoreSes, TypStoreSes);
+
+            Type curType = type;
+
+            while (curType != null)
+            {
+                CreateInjectors(curType, gridCollector, storeSesCollector);
+
+                curType = curType.BaseType;
+            }
+
+            _igniteInjectors = gridCollector.Injectors;
+            _storeSesInjectors = storeSesCollector.Injectors;
+
+            _taskNoResCache = ContainsAttribute(type, TypComputeTaskNoResCache, true);
+        }
+
+        /// <summary>
+        /// Inject resources to the given object.
+        /// </summary>
+        /// <param name="target">Target.</param>
+        /// <param name="ignite">Grid.</param>
+        public void InjectIgnite(object target, Ignite ignite)
+        {
+            InjectIgnite(target, ignite.Proxy);
+        }
+
+        /// <summary>
+        /// Inject resources to the given object.
+        /// </summary>
+        /// <param name="target">Target.</param>
+        /// <param name="igniteProxy">Grid proxy.</param>
+        public void InjectIgnite(object target, IgniteProxy igniteProxy)
+        {
+            Inject0(target, igniteProxy, _igniteInjectors);
+        }
+
+        /// <summary>
+        /// Inject store session.
+        /// </summary>
+        /// <param name="target">Target.</param>
+        /// <param name="ses">Store session.</param>
+        public void InjectStoreSession(object target, ICacheStoreSession ses)
+        {
+            Inject0(target, ses, _storeSesInjectors);
+        }
+
+        /// <summary>
+        /// Perform injection.
+        /// </summary>
+        /// <param name="target">Target.</param>
+        /// <param name="injectee">Injectee.</param>
+        /// <param name="injectors">Injectors.</param>
+        private static void Inject0(object target, object injectee, ICollection<IResourceInjector> injectors)
+        {
+            if (injectors != null)
+            {
+                foreach (IResourceInjector injector in injectors)
+                    injector.Inject(target, injectee);    
+            }
+        }
+
+        /// <summary>
+        /// Task "no result cache" flag.
+        /// </summary>
+        public bool TaskNoResultCache
+        {
+            get
+            {
+                return _taskNoResCache;
+            }
+        }
+        
+        /// <summary>
+        /// Create gridInjectors for the given type.
+        /// </summary>
+        /// <param name="type">Type.</param>
+        /// <param name="collectors">Collectors.</param>
+        private static void CreateInjectors(Type type, params Collector[] collectors)
+        {
+            FieldInfo[] fields = type.GetFields(Flags);
+
+            foreach (FieldInfo field in fields)
+            {
+                foreach (var collector in collectors)
+                {
+                    if (!ContainsAttribute(field, collector.AttributeType, false))
+                        continue;
+
+                    if (!field.FieldType.IsAssignableFrom(collector.ResourceType))
+                        throw new IgniteException("Invalid field type for resource attribute [" + 
+                            "type=" + type.Name +
+                            ", field=" + field.Name + 
+                            ", fieldType=" + field.FieldType.Name + 
+                            ", resourceType=" + collector.ResourceType.Name + ']');
+
+                    collector.Add(new ResourceFieldInjector(field));
+                }
+            }
+
+            PropertyInfo[] props = type.GetProperties(Flags);
+
+            foreach (var prop in props)
+            {
+                foreach (var collector in collectors)
+                {
+                    if (!ContainsAttribute(prop, collector.AttributeType, false))
+                        continue;
+
+                    if (!prop.CanWrite)
+                        throw new IgniteException("Property with resource attribute is not writable [" +
+                            "type=" + type.Name + 
+                            ", property=" + prop.Name +
+                            ", resourceType=" + collector.ResourceType.Name + ']');
+
+                    if (!prop.PropertyType.IsAssignableFrom(collector.ResourceType))
+                        throw new IgniteException("Invalid property type for resource attribute [" + 
+                            "type=" + type.Name +
+                            ", property=" + prop.Name + 
+                            ", propertyType=" + prop.PropertyType.Name + 
+                            ", resourceType=" + collector.ResourceType.Name + ']');
+
+                    collector.Add(new ResourcePropertyInjector(prop));
+                }
+            }
+
+            MethodInfo[] mthds = type.GetMethods(Flags);
+
+            foreach (MethodInfo mthd in mthds)
+            {
+                foreach (var collector in collectors)
+                {
+                    if (!ContainsAttribute(mthd, collector.AttributeType, false)) 
+                        continue;
+
+                    ParameterInfo[] parameters = mthd.GetParameters();
+
+                    if (parameters.Length != 1)
+                        throw new IgniteException("Method with resource attribute must have only one parameter [" + 
+                            "type=" + type.Name + 
+                            ", method=" + mthd.Name +
+                            ", resourceType=" + collector.ResourceType.Name + ']');
+
+                    if (!parameters[0].ParameterType.IsAssignableFrom(collector.ResourceType))
+                        throw new IgniteException("Invalid method parameter type for resource attribute [" +
+                            "type=" + type.Name + 
+                            ", method=" + mthd.Name + 
+                            ", methodParameterType=" + parameters[0].ParameterType.Name + 
+                            ", resourceType=" + collector.ResourceType.Name + ']');
+
+                    collector.Add(new ResourceMethodInjector(mthd));
+                }
+            }
+        }
+        
+        /// <summary>
+        /// Check whether the given member contains the given attribute.
+        /// </summary>
+        /// <param name="member">Mmeber.</param>
+        /// <param name="attrType">Attribute type.</param>
+        /// <param name="inherit">Inherit flag.</param>
+        /// <returns>True if contains</returns>
+        private static bool ContainsAttribute(MemberInfo member, Type attrType, bool inherit)
+        {
+            return member.GetCustomAttributes(attrType, inherit).Length > 0;
+        }
+
+        /// <summary>
+        /// Collector.
+        /// </summary>
+        private class Collector
+        {
+            /** Attribute type. */
+            private readonly Type _attrType;
+
+            /** Resource type. */
+            private readonly Type _resType;
+            
+            /// <summary>
+            /// Constructor.
+            /// </summary>
+            /// <param name="attrType">Atrribute type.</param>
+            /// <param name="resType">Resource type.</param>
+            public Collector(Type attrType, Type resType)
+            {
+                _attrType = attrType;
+                _resType = resType;
+            }
+
+            /// <summary>
+            /// Attribute type.
+            /// </summary>
+            public Type AttributeType
+            {
+                get { return _attrType; }
+            }
+
+            /// <summary>
+            /// Resource type.
+            /// </summary>
+            public Type ResourceType
+            {
+                get { return _resType; }
+            }
+
+            /// <summary>
+            /// Add injector.
+            /// </summary>
+            /// <param name="injector">Injector.</param>
+            public void Add(IResourceInjector injector)
+            {
+                if (Injectors == null)
+                    Injectors = new List<IResourceInjector> { injector };
+                else
+                    Injectors.Add(injector);
+            }
+
+            /// <summary>
+            /// Injectors.
+            /// </summary>
+            public List<IResourceInjector> Injectors { get; private set; }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/ServiceContext.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/ServiceContext.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/ServiceContext.cs
new file mode 100644
index 0000000..f5674f3
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/ServiceContext.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.Services
+{
+    using System;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Portable;
+    using Apache.Ignite.Core.Services;
+
+    /// <summary>
+    /// Service context.
+    /// </summary>
+    internal class ServiceContext : IServiceContext
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ServiceContext"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        public ServiceContext(IPortableRawReader reader)
+        {
+            Debug.Assert(reader != null);
+
+            Name = reader.ReadString();
+            ExecutionId = reader.ReadGuid() ?? Guid.Empty;
+            IsCancelled = reader.ReadBoolean();
+            CacheName = reader.ReadString();
+            AffinityKey = reader.ReadObject<object>();
+        }
+
+        /** <inheritdoc /> */
+        public string Name { get; private set; }
+
+        /** <inheritdoc /> */
+        public Guid ExecutionId { get; private set; }
+
+        /** <inheritdoc /> */
+        public bool IsCancelled { get; private set; }
+
+        /** <inheritdoc /> */
+        public string CacheName { get; private set; }
+
+        /** <inheritdoc /> */
+        public object AffinityKey { get; private set; }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/ServiceDescriptor.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/ServiceDescriptor.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/ServiceDescriptor.cs
new file mode 100644
index 0000000..9bd9814
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/ServiceDescriptor.cs
@@ -0,0 +1,106 @@
+/*
+ * 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.Services
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Impl.Collections;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Services;
+
+    /// <summary>
+    /// Service descriptor.
+    /// </summary>
+    internal class ServiceDescriptor : IServiceDescriptor
+    {
+        /** Services. */
+        private readonly IServices _services;
+
+        /** Service type. */
+        private Type _type;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ServiceDescriptor" /> class.
+        /// </summary>
+        /// <param name="name">Name.</param>
+        /// <param name="reader">Reader.</param>
+        /// <param name="services">Services.</param>
+        public ServiceDescriptor(string name, PortableReaderImpl reader, IServices services)
+        {
+            Debug.Assert(reader != null);
+            Debug.Assert(services != null);
+            Debug.Assert(!string.IsNullOrEmpty(name));
+
+            _services = services;
+            Name = name;
+
+            CacheName = reader.ReadString();
+            MaxPerNodeCount = reader.ReadInt();
+            TotalCount = reader.ReadInt();
+            OriginNodeId = reader.ReadGuid() ?? Guid.Empty;
+            AffinityKey = reader.ReadObject<object>();
+
+            var mapSize = reader.ReadInt();
+            var snap = new Dictionary<Guid, int>(mapSize);
+
+            for (var i = 0; i < mapSize; i++)
+                snap[reader.ReadGuid() ?? Guid.Empty] = reader.ReadInt();
+
+            TopologySnapshot = snap.AsReadOnly();
+        }
+
+        /** <inheritdoc /> */
+        public string Name { get; private set; }
+        
+        /** <inheritdoc /> */
+        public Type Type
+        {
+            get
+            {
+                try
+                {
+                    return _type ?? (_type = _services.GetServiceProxy<IService>(Name).GetType());
+                }
+                catch (Exception ex)
+                {
+                    throw new ServiceInvocationException(
+                        "Failed to retrieve service type. It has either been cancelled, or is not a .Net service", ex);
+                }
+            }
+        }
+
+        /** <inheritdoc /> */
+        public int TotalCount { get; private set; }
+
+        /** <inheritdoc /> */
+        public int MaxPerNodeCount { get; private set; }
+
+        /** <inheritdoc /> */
+        public string CacheName { get; private set; }
+
+        /** <inheritdoc /> */
+        public object AffinityKey { get; private set; }
+
+        /** <inheritdoc /> */
+        public Guid OriginNodeId { get; private set; }
+
+        /** <inheritdoc /> */
+        public IDictionary<Guid, int> TopologySnapshot { get; private set; }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/ServiceProxy.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/ServiceProxy.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/ServiceProxy.cs
new file mode 100644
index 0000000..ebb4c84
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/ServiceProxy.cs
@@ -0,0 +1,71 @@
+/*
+ * 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.Services
+{
+    using System;
+    using System.Diagnostics;
+    using System.Reflection;
+    using System.Runtime.Remoting.Messaging;
+    using System.Runtime.Remoting.Proxies;
+
+    /// <summary>
+    /// Service proxy: user works with a remote service as if it is a local object.
+    /// </summary>
+    /// <typeparam name="T">User type to be proxied.</typeparam>
+    internal class ServiceProxy<T> : RealProxy
+    {
+        /** Services. */
+        private readonly Func<MethodBase, object[], object> _invokeAction;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ServiceProxy{T}" /> class.
+        /// </summary>
+        /// <param name="invokeAction">Method invoke action.</param>
+        public ServiceProxy(Func<MethodBase, object[], object> invokeAction)
+            : base(typeof (T))
+        {
+            Debug.Assert(invokeAction != null);
+
+            _invokeAction = invokeAction;
+        }
+
+        /** <inheritdoc /> */
+        public override IMessage Invoke(IMessage msg)
+        {
+            var methodCall = msg as IMethodCallMessage;
+
+            if (methodCall == null)
+                throw new NotSupportedException("Service proxy operation type not supported: " + msg.GetType() +
+                                                ". Only method and property calls are supported.");
+
+            if (methodCall.InArgCount != methodCall.ArgCount)
+                throw new NotSupportedException("Service proxy does not support out arguments: "
+                                                + methodCall.MethodBase);
+
+            var result = _invokeAction(methodCall.MethodBase, methodCall.Args);
+
+            return new ReturnMessage(result, null, 0, methodCall.LogicalCallContext, methodCall);
+        }
+
+        /** <inheritdoc /> */
+        public new T GetTransparentProxy()
+        {
+            return (T) base.GetTransparentProxy();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/ServiceProxyInvoker.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/ServiceProxyInvoker.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/ServiceProxyInvoker.cs
new file mode 100644
index 0000000..fa5da17
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/ServiceProxyInvoker.cs
@@ -0,0 +1,136 @@
+/*
+ * 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.Services
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using System.Linq;
+    using System.Reflection;
+
+    /// <summary>
+    /// Invokes service proxy methods.
+    /// </summary>
+    internal static class ServiceProxyInvoker 
+    {
+        /// <summary>
+        /// Invokes the service method according to data from a stream,
+        /// and writes invocation result to the output stream.
+        /// </summary>
+        /// <param name="svc">Service instance.</param>
+        /// <param name="methodName">Name of the method.</param>
+        /// <param name="arguments">Arguments.</param>
+        /// <returns>Pair of method return value and invocation exception.</returns>
+        public static KeyValuePair<object, Exception> InvokeServiceMethod(object svc, string methodName, 
+            object[] arguments)
+        {
+            Debug.Assert(svc != null);
+            Debug.Assert(!string.IsNullOrWhiteSpace(methodName));
+
+            var method = GetMethodOrThrow(svc.GetType(), methodName, arguments);
+
+            try
+            {
+                return new KeyValuePair<object, Exception>(method.Invoke(svc, arguments), null);
+            }
+            catch (TargetInvocationException invokeErr)
+            {
+                return new KeyValuePair<object, Exception>(null, invokeErr.InnerException);
+            }
+            catch (Exception err)
+            {
+                return new KeyValuePair<object, Exception>(null, err);
+            }
+        }
+
+        /// <summary>
+        /// Finds suitable method in the specified type, or throws an exception.
+        /// </summary>
+        private static MethodBase GetMethodOrThrow(Type svcType, string methodName, object[] arguments)
+        {
+            Debug.Assert(svcType != null);
+            Debug.Assert(!string.IsNullOrWhiteSpace(methodName));
+
+            // 1) Find methods by name
+            var methods = svcType.GetMethods(BindingFlags.Public | BindingFlags.NonPublic | BindingFlags.Instance)
+                .Where(m => CleanupMethodName(m) == methodName).ToArray();
+
+            if (methods.Length == 1)
+                return methods[0];
+
+            if (methods.Length == 0)
+                throw new InvalidOperationException(
+                    string.Format("Failed to invoke proxy: there is no method '{0}' in type '{1}'", 
+                    methodName, svcType));
+
+            // 2) There is more than 1 method with specified name - resolve with argument types.
+            methods = methods.Where(m => AreMethodArgsCompatible(arguments, m.GetParameters())).ToArray();
+
+            if (methods.Length == 1)
+                return methods[0];
+
+            // 3) 0 or more than 1 matching method - throw.
+            var argsString = arguments == null || arguments.Length == 0
+                ? "0"
+                : "(" +
+                  arguments.Select(x => x == null ? "null" : x.GetType().Name).Aggregate((x, y) => x + ", " + y)
+                  + ")";
+
+            if (methods.Length == 0)
+                throw new InvalidOperationException(
+                    string.Format("Failed to invoke proxy: there is no method '{0}' in type '{1}' with {2} arguments",
+                    methodName, svcType, argsString));
+
+            throw new InvalidOperationException(
+                string.Format("Failed to invoke proxy: there are {2} methods '{0}' in type '{1}' with {3} " +
+                              "arguments, can't resolve ambiguity.", methodName, svcType, methods.Length, argsString));
+        }
+        
+        /// <summary>
+        /// Cleans up a method name by removing interface part, 
+        /// which occurs when explicit interface implementation is used.
+        /// </summary>
+        private static string CleanupMethodName(MethodBase method)
+        {
+            var name = method.Name;
+
+            var dotIdx = name.LastIndexOf(Type.Delimiter);
+
+            return dotIdx < 0 ? name : name.Substring(dotIdx + 1);
+        }
+
+        /// <summary>
+        /// Determines whether specified method arguments are comatible with given method parameter definitions.
+        /// </summary>
+        /// <param name="methodArgs">Method argument types.</param>
+        /// <param name="targetParameters">Target method parameter definitions.</param>
+        /// <returns>True if a target method can be called with specified set of arguments; otherwise, false.</returns>
+        private static bool AreMethodArgsCompatible(object[] methodArgs, ParameterInfo[] targetParameters)
+        {
+            if (methodArgs == null || methodArgs.Length == 0)
+                return targetParameters.Length == 0;
+
+            if (methodArgs.Length != targetParameters.Length)
+                return false;
+
+            return methodArgs
+                .Zip(targetParameters, (arg, param) => arg == null || param.ParameterType.IsInstanceOfType(arg))
+                .All(x => x);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/ServiceProxySerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/ServiceProxySerializer.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/ServiceProxySerializer.cs
new file mode 100644
index 0000000..e7af8da
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/ServiceProxySerializer.cs
@@ -0,0 +1,140 @@
+/*
+ * 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.Services
+{
+    using System;
+    using System.Diagnostics;
+    using System.Reflection;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Portable;
+    using Apache.Ignite.Core.Services;
+
+    /// <summary>
+    /// Static proxy methods.
+    /// </summary>
+    internal static class ServiceProxySerializer
+    {
+        /// <summary>
+        /// Writes proxy method invocation data to the specified writer.
+        /// </summary>
+        /// <param name="writer">Writer.</param>
+        /// <param name="method">Method.</param>
+        /// <param name="arguments">Arguments.</param>
+        public static void WriteProxyMethod(PortableWriterImpl writer, MethodBase method, object[] arguments)
+        {
+            Debug.Assert(writer != null);
+            Debug.Assert(method != null);
+
+            writer.WriteString(method.Name);
+
+            if (arguments != null)
+            {
+                writer.WriteBoolean(true);
+                writer.WriteInt(arguments.Length);
+
+                foreach (var arg in arguments)
+                    writer.WriteObject(arg);
+            }
+            else
+                writer.WriteBoolean(false);
+        }
+
+        /// <summary>
+        /// Reads proxy method invocation data from the specified reader.
+        /// </summary>
+        /// <param name="stream">Stream.</param>
+        /// <param name="marsh">Marshaller.</param>
+        /// <param name="mthdName">Method name.</param>
+        /// <param name="mthdArgs">Method arguments.</param>
+        public static void ReadProxyMethod(IPortableStream stream, PortableMarshaller marsh, 
+            out string mthdName, out object[] mthdArgs)
+        {
+            var reader = marsh.StartUnmarshal(stream);
+
+            var srvKeepPortable = reader.ReadBoolean();
+
+            mthdName = reader.ReadString();
+
+            if (reader.ReadBoolean())
+            {
+                mthdArgs = new object[reader.ReadInt()];
+
+                if (srvKeepPortable)
+                    reader = marsh.StartUnmarshal(stream, true);
+
+                for (var i = 0; i < mthdArgs.Length; i++)
+                    mthdArgs[i] = reader.ReadObject<object>();
+            }
+            else
+                mthdArgs = null;
+        }
+
+        /// <summary>
+        /// Writes method invocation result.
+        /// </summary>
+        /// <param name="stream">Stream.</param>
+        /// <param name="marsh">Marshaller.</param>
+        /// <param name="methodResult">Method result.</param>
+        /// <param name="invocationError">Method invocation error.</param>
+        public static void WriteInvocationResult(IPortableStream stream, PortableMarshaller marsh, object methodResult,
+            Exception invocationError)
+        {
+            Debug.Assert(stream != null);
+            Debug.Assert(marsh != null);
+
+            var writer = marsh.StartMarshal(stream);
+
+            PortableUtils.WriteInvocationResult(writer, invocationError == null, invocationError ?? methodResult);
+        }
+
+        /// <summary>
+        /// Reads method invocation result.
+        /// </summary>
+        /// <param name="stream">Stream.</param>
+        /// <param name="marsh">Marshaller.</param>
+        /// <param name="keepPortable">Portable flag.</param>
+        /// <returns>
+        /// Method invocation result, or exception in case of error.
+        /// </returns>
+        public static object ReadInvocationResult(IPortableStream stream, PortableMarshaller marsh, bool keepPortable)
+        {
+            Debug.Assert(stream != null);
+            Debug.Assert(marsh != null);
+
+            var mode = keepPortable ? PortableMode.ForcePortable : PortableMode.Deserialize;
+
+            var reader = marsh.StartUnmarshal(stream, mode);
+
+            object err;
+
+            var res = PortableUtils.ReadInvocationResult(reader, out err);
+
+            if (err == null)
+                return res;
+
+            var portErr = err as IPortableObject;
+
+            throw portErr != null
+                ? new ServiceInvocationException("Proxy method invocation failed with a portable error. " +
+                                                 "Examine PortableCause for details.", portErr)
+                : new ServiceInvocationException("Proxy method invocation failed with an exception. " +
+                                                 "Examine InnerException for details.", (Exception) err);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs
new file mode 100644
index 0000000..38a7175
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs
@@ -0,0 +1,316 @@
+/*
+ * 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.Services
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using System.Linq;
+    using System.Reflection;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+    using Apache.Ignite.Core.Services;
+    using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils;
+
+    /// <summary>
+    /// Services implementation.
+    /// </summary>
+    internal class Services : PlatformTarget, IServices
+    {
+        /** */
+        private const int OpDeploy = 1;
+        
+        /** */
+        private const int OpDeployMultiple = 2;
+
+        /** */
+        private const int OpDotnetServices = 3;
+
+        /** */
+        private const int OpInvokeMethod = 4;
+
+        /** */
+        private const int OpDescriptors = 5;
+
+        /** */
+        private readonly IClusterGroup _clusterGroup;
+
+        /** Invoker portable flag. */
+        protected readonly bool KeepPortable;
+
+        /** Server portable flag. */
+        protected readonly bool SrvKeepPortable;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="Services" /> class.
+        /// </summary>
+        /// <param name="target">Target.</param>
+        /// <param name="marsh">Marshaller.</param>
+        /// <param name="clusterGroup">Cluster group.</param>
+        /// <param name="keepPortable">Invoker portable flag.</param>
+        /// <param name="srvKeepPortable">Server portable flag.</param>
+        public Services(IUnmanagedTarget target, PortableMarshaller marsh, IClusterGroup clusterGroup, 
+            bool keepPortable, bool srvKeepPortable)
+            : base(target, marsh)
+        {
+            Debug.Assert(clusterGroup  != null);
+
+            _clusterGroup = clusterGroup;
+            KeepPortable = keepPortable;
+            SrvKeepPortable = srvKeepPortable;
+        }
+
+        /** <inheritDoc /> */
+        public virtual IServices WithKeepPortable()
+        {
+            if (KeepPortable)
+                return this;
+
+            return new Services(Target, Marshaller, _clusterGroup, true, SrvKeepPortable);
+        }
+
+        /** <inheritDoc /> */
+        public virtual IServices WithServerKeepPortable()
+        {
+            if (SrvKeepPortable)
+                return this;
+
+            return new Services(UU.ServicesWithServerKeepPortable(Target), Marshaller, _clusterGroup, KeepPortable, true);
+        }
+
+        /** <inheritDoc /> */
+        public virtual IServices WithAsync()
+        {
+            return new ServicesAsync(UU.ServicesWithAsync(Target), Marshaller, _clusterGroup, KeepPortable, SrvKeepPortable);
+        }
+
+        /** <inheritDoc /> */
+        public virtual bool IsAsync
+        {
+            get { return false; }
+        }
+
+        /** <inheritDoc /> */
+        public virtual IFuture GetFuture()
+        {
+            throw new InvalidOperationException("Asynchronous mode is disabled");
+        }
+
+        /** <inheritDoc /> */
+        public virtual IFuture<TResult> GetFuture<TResult>()
+        {
+            throw new InvalidOperationException("Asynchronous mode is disabled");
+        }
+
+        /** <inheritDoc /> */
+        public IClusterGroup ClusterGroup
+        {
+            get { return _clusterGroup; }
+        }
+
+        /** <inheritDoc /> */
+        public void DeployClusterSingleton(string name, IService service)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(name, "name");
+            IgniteArgumentCheck.NotNull(service, "service");
+
+            DeployMultiple(name, service, 1, 1);
+        }
+
+        /** <inheritDoc /> */
+        public void DeployNodeSingleton(string name, IService service)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(name, "name");
+            IgniteArgumentCheck.NotNull(service, "service");
+
+            DeployMultiple(name, service, 0, 1);
+        }
+
+        /** <inheritDoc /> */
+        public void DeployKeyAffinitySingleton<TK>(string name, IService service, string cacheName, TK affinityKey)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(name, "name");
+            IgniteArgumentCheck.NotNull(service, "service");
+            IgniteArgumentCheck.NotNull(affinityKey, "affinityKey");
+
+            Deploy(new ServiceConfiguration
+            {
+                Name = name,
+                Service = service,
+                CacheName = cacheName,
+                AffinityKey = affinityKey,
+                TotalCount = 1,
+                MaxPerNodeCount = 1
+            });
+        }
+
+        /** <inheritDoc /> */
+        public void DeployMultiple(string name, IService service, int totalCount, int maxPerNodeCount)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(name, "name");
+            IgniteArgumentCheck.NotNull(service, "service");
+
+            DoOutOp(OpDeployMultiple, w =>
+            {
+                w.WriteString(name);
+                w.WriteObject(service);
+                w.WriteInt(totalCount);
+                w.WriteInt(maxPerNodeCount);
+            });
+        }
+
+        /** <inheritDoc /> */
+        public void Deploy(ServiceConfiguration configuration)
+        {
+            IgniteArgumentCheck.NotNull(configuration, "configuration");
+
+            DoOutOp(OpDeploy, w =>
+            {
+                w.WriteString(configuration.Name);
+                w.WriteObject(configuration.Service);
+                w.WriteInt(configuration.TotalCount);
+                w.WriteInt(configuration.MaxPerNodeCount);
+                w.WriteString(configuration.CacheName);
+                w.WriteObject(configuration.AffinityKey);
+
+                if (configuration.NodeFilter != null)
+                    w.WriteObject(new PortableOrSerializableObjectHolder(configuration.NodeFilter));
+                else
+                    w.WriteObject<PortableOrSerializableObjectHolder>(null);
+            });
+        }
+
+        /** <inheritDoc /> */
+        public void Cancel(string name)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(name, "name");
+
+            UU.ServicesCancel(Target, name);
+        }
+
+        /** <inheritDoc /> */
+        public void CancelAll()
+        {
+            UU.ServicesCancelAll(Target);
+        }
+
+        /** <inheritDoc /> */
+        public ICollection<IServiceDescriptor> GetServiceDescriptors()
+        {
+            return DoInOp(OpDescriptors, stream =>
+            {
+                var reader = Marshaller.StartUnmarshal(stream, KeepPortable);
+
+                var size = reader.ReadInt();
+
+                var result = new List<IServiceDescriptor>(size);
+
+                for (var i = 0; i < size; i++)
+                {
+                    var name = reader.ReadString();
+
+                    result.Add(new ServiceDescriptor(name, reader, this));
+                }
+
+                return result;
+            });
+        }
+
+        /** <inheritDoc /> */
+        public T GetService<T>(string name)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(name, "name");
+
+            var services = GetServices<T>(name);
+
+            if (services == null)
+                return default(T);
+
+            return services.FirstOrDefault();
+        }
+
+        /** <inheritDoc /> */
+        public ICollection<T> GetServices<T>(string name)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(name, "name");
+
+            return DoOutInOp<ICollection<T>>(OpDotnetServices, w => w.WriteString(name),
+                r =>
+                {
+                    bool hasVal = r.ReadBool();
+
+                    if (hasVal)
+                    {
+                        var count = r.ReadInt();
+                        
+                        var res = new List<T>(count);
+
+                        for (var i = 0; i < count; i++)
+                            res.Add((T)Marshaller.Ignite.HandleRegistry.Get<IService>(r.ReadLong()));
+
+                        return res;
+                    }
+                    return null;
+                });
+        }
+
+        /** <inheritDoc /> */
+        public T GetServiceProxy<T>(string name) where T : class
+        {
+            return GetServiceProxy<T>(name, false);
+        }
+
+        /** <inheritDoc /> */
+        public T GetServiceProxy<T>(string name, bool sticky) where T : class
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(name, "name");
+            IgniteArgumentCheck.Ensure(typeof(T).IsInterface, "T", "Service proxy type should be an interface: " + typeof(T));
+
+            // In local scenario try to return service instance itself instead of a proxy
+            // Get as object because proxy interface may be different from real interface
+            var locInst = GetService<object>(name) as T;
+
+            if (locInst != null)
+                return locInst;
+
+            var javaProxy = UU.ServicesGetServiceProxy(Target, name, sticky);
+
+            return new ServiceProxy<T>((method, args) => InvokeProxyMethod(javaProxy, method, args))
+                .GetTransparentProxy();
+        }
+
+        /// <summary>
+        /// Invokes the service proxy method.
+        /// </summary>
+        /// <param name="proxy">Unmanaged proxy.</param>
+        /// <param name="method">Method to invoke.</param>
+        /// <param name="args">Arguments.</param>
+        /// <returns>
+        /// Invocation result.
+        /// </returns>
+        private unsafe object InvokeProxyMethod(IUnmanagedTarget proxy, MethodBase method, object[] args)
+        {
+            return DoOutInOp(OpInvokeMethod,
+                writer => ServiceProxySerializer.WriteProxyMethod(writer, method, args),
+                stream => ServiceProxySerializer.ReadInvocationResult(stream, Marshaller, KeepPortable), proxy.Target);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/ServicesAsync.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/ServicesAsync.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/ServicesAsync.cs
new file mode 100644
index 0000000..860de45
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Services/ServicesAsync.cs
@@ -0,0 +1,89 @@
+/*
+ * 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.Services
+{
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+    using Apache.Ignite.Core.Services;
+    using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils;
+
+    /// <summary>
+    /// Async services implementation.
+    /// </summary>
+    internal class ServicesAsync : Services
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ServicesAsync" /> class.
+        /// </summary>
+        /// <param name="target">Target.</param>
+        /// <param name="marsh">Marshaller.</param>
+        /// <param name="clusterGroup">Cluster group.</param>
+        /// <param name="keepPortable">Portable flag.</param>
+        /// <param name="srvKeepPortable">Server portable flag.</param>
+        public ServicesAsync(IUnmanagedTarget target, PortableMarshaller marsh, IClusterGroup clusterGroup,
+            bool keepPortable, bool srvKeepPortable)
+            : base(target, marsh, clusterGroup, keepPortable, srvKeepPortable)
+        {
+            // No-op
+        }
+
+        /** <inheritDoc /> */
+        public override bool IsAsync
+        {
+            get { return true; }
+        }
+
+        /** <inheritDoc /> */
+        public override IServices WithKeepPortable()
+        {
+            if (KeepPortable)
+                return this;
+
+            return new ServicesAsync(Target, Marshaller, ClusterGroup, true, SrvKeepPortable);
+        }
+
+        /** <inheritDoc /> */
+        public override IServices WithServerKeepPortable()
+        {
+            if (SrvKeepPortable)
+                return this;
+
+            return new ServicesAsync(Target, Marshaller, ClusterGroup, KeepPortable, true);
+        }
+
+        /** <inheritDoc /> */
+        public override IServices WithAsync()
+        {
+            return this;
+        }
+
+        /** <inheritDoc /> */
+        public override IFuture GetFuture()
+        {
+            return GetFuture<object>();
+        }
+
+        /** <inheritDoc /> */
+        public override IFuture<T> GetFuture<T>()
+        {
+            return GetFuture<T>((futId, futTyp) => UU.TargetListenFuture(Target, futId, futTyp));
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Transactions/AsyncTransaction.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Transactions/AsyncTransaction.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Transactions/AsyncTransaction.cs
new file mode 100644
index 0000000..82d1d55
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Transactions/AsyncTransaction.cs
@@ -0,0 +1,78 @@
+/*
+ * 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.Transactions
+{
+    using System;
+    using System.Threading;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// Grid async transaction facade.
+    /// </summary>
+    internal class AsyncTransaction : Transaction
+    {
+        /** */
+        private readonly ThreadLocal<IFuture> _curFut = new ThreadLocal<IFuture>();
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AsyncTransaction"/> class.
+        /// </summary>
+        /// <param name="tx">The tx to wrap.</param>
+        public AsyncTransaction(TransactionImpl tx) : base(tx)
+        {
+            // No-op.
+        }
+
+        /** <inheritDoc /> */
+        public override bool IsAsync
+        {
+            get { return true; }
+        }
+
+        /** <inheritDoc /> */
+        public override IFuture<TResult> GetFuture<TResult>()
+        {
+            return GetFuture() as IFuture<TResult>;
+        }
+
+        /** <inheritDoc /> */
+        public override IFuture GetFuture()
+        {
+            var fut = _curFut.Value;
+
+            if (fut == null)
+                throw new InvalidOperationException("Asynchronous operation not started.");
+
+            _curFut.Value = null;
+
+            return fut;
+        }
+
+        /** <inheritDoc /> */
+        public override void Commit()
+        {
+            _curFut.Value = Tx.GetFutureOrError(() => Tx.CommitAsync());
+        }
+
+        /** <inheritDoc /> */
+        public override void Rollback()
+        {
+            _curFut.Value = Tx.GetFutureOrError(() => Tx.RollbackAsync());
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Transactions/Transaction.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Transactions/Transaction.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Transactions/Transaction.cs
new file mode 100644
index 0000000..47c9f93
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Transactions/Transaction.cs
@@ -0,0 +1,155 @@
+/*
+ * 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.Transactions
+{
+    using System;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Transactions;
+
+    /// <summary>
+    /// Ignite transaction facade.
+    /// </summary>
+    internal class Transaction : ITransaction
+    {
+        /** */
+        protected readonly TransactionImpl Tx;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="Transaction" /> class.
+        /// </summary>
+        /// <param name="tx">The tx to wrap.</param>
+        public Transaction(TransactionImpl tx)
+        {
+            Tx = tx;
+        }
+
+        /** <inheritDoc /> */
+        public void Dispose()
+        {
+            Tx.Dispose();
+        }
+
+        /** <inheritDoc /> */
+        public ITransaction WithAsync()
+        {
+            return new AsyncTransaction(Tx);
+        }
+
+        /** <inheritDoc /> */
+        public virtual bool IsAsync
+        {
+            get { return false; }
+        }
+
+        /** <inheritDoc /> */
+        public virtual IFuture GetFuture()
+        {
+            throw IgniteUtils.GetAsyncModeDisabledException();
+        }
+        
+        /** <inheritDoc /> */
+        public virtual IFuture<TResult> GetFuture<TResult>()
+        {
+            throw IgniteUtils.GetAsyncModeDisabledException();
+        }
+
+        /** <inheritDoc /> */
+        public Guid NodeId
+        {
+            get { return Tx.NodeId; }
+        }
+
+        /** <inheritDoc /> */
+        public long ThreadId
+        {
+            get { return Tx.ThreadId; }
+        }
+
+        /** <inheritDoc /> */
+        public DateTime StartTime
+        {
+            get { return Tx.StartTime; }
+        }
+
+        /** <inheritDoc /> */
+        public TransactionIsolation Isolation
+        {
+            get { return Tx.Isolation; }
+        }
+
+        /** <inheritDoc /> */
+        public TransactionConcurrency Concurrency
+        {
+            get { return Tx.Concurrency; }
+        }
+
+        /** <inheritDoc /> */
+        public TransactionState State
+        {
+            get { return Tx.State; }
+        }
+
+        /** <inheritDoc /> */
+        public TimeSpan Timeout
+        {
+            get { return Tx.Timeout; }
+        }
+
+        /** <inheritDoc /> */
+        public bool IsRollbackOnly
+        {
+            get { return Tx.IsRollbackOnly; }
+        }
+
+        /** <inheritDoc /> */
+        public bool SetRollbackonly()
+        {
+            return Tx.SetRollbackOnly();
+        }
+
+        /** <inheritDoc /> */
+        public virtual void Commit()
+        {
+            Tx.Commit();
+        }
+
+        /** <inheritDoc /> */
+        public virtual void Rollback()
+        {
+            Tx.Rollback();
+        }
+
+        /** <inheritDoc /> */
+        public void AddMeta<TV>(string name, TV val)
+        {
+            Tx.AddMeta(name, val);
+        }
+
+        /** <inheritDoc /> */
+        public TV Meta<TV>(string name)
+        {
+            return Tx.Meta<TV>(name);
+        }
+
+        /** <inheritDoc /> */
+        public TV RemoveMeta<TV>(string name)
+        {
+            return Tx.RemoveMeta<TV>(name);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionImpl.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionImpl.cs
new file mode 100644
index 0000000..9e71181
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionImpl.cs
@@ -0,0 +1,489 @@
+/*
+ * 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.Transactions
+{
+    using System;
+    using System.Threading;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Transactions;
+
+    /// <summary>
+    /// Grid cache transaction implementation.
+    /// </summary>
+    internal sealed class TransactionImpl
+    {
+        /** Metadatas. */
+        private object[] _metas;
+
+        /** Unique  transaction ID.*/
+        private readonly long _id;
+
+        /** Cache. */
+        private readonly TransactionsImpl _txs;
+
+        /** TX concurrency. */
+        private readonly TransactionConcurrency _concurrency;
+
+        /** TX isolation. */
+        private readonly TransactionIsolation _isolation;
+
+        /** Timeout. */
+        private readonly TimeSpan _timeout;
+
+        /** Start time. */
+        private readonly DateTime _startTime;
+
+        /** Owning thread ID. */
+        private readonly int _threadId;
+
+        /** Originating node ID. */
+        private readonly Guid _nodeId;
+
+        /** State holder. */
+        private StateHolder _state;
+
+        // ReSharper disable once InconsistentNaming
+        /** Transaction for this thread. */
+        [ThreadStatic]
+        private static TransactionImpl THREAD_TX;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="id">ID.</param>
+        /// <param name="txs">Transactions.</param>
+        /// <param name="concurrency">TX concurrency.</param>
+        /// <param name="isolation">TX isolation.</param>
+        /// <param name="timeout">Timeout.</param>
+        /// <param name="nodeId">The originating node identifier.</param>
+        public TransactionImpl(long id, TransactionsImpl txs, TransactionConcurrency concurrency,
+            TransactionIsolation isolation, TimeSpan timeout, Guid nodeId) {
+            _id = id;
+            _txs = txs;
+            _concurrency = concurrency;
+            _isolation = isolation;
+            _timeout = timeout;
+            _nodeId = nodeId;
+
+            _startTime = DateTime.Now;
+
+            _threadId = Thread.CurrentThread.ManagedThreadId;
+
+            THREAD_TX = this;
+        }    
+
+        /// <summary>
+        /// Transaction assigned to this thread.
+        /// </summary>
+        public static Transaction Current
+        {
+            get
+            {
+                var tx = THREAD_TX;
+
+                if (tx == null)
+                    return null;
+
+                if (tx.IsClosed)
+                {
+                    THREAD_TX = null;
+
+                    return null;
+                }
+
+                return new Transaction(tx);
+            }
+        }
+
+        /// <summary>
+        /// Commits this tx and closes it.
+        /// </summary>
+        public void Commit()
+        {
+            lock (this)
+            {
+                ThrowIfClosed();
+
+                _state = new StateHolder(_txs.TxCommit(this));
+            }
+        }
+
+        /// <summary>
+        /// Rolls this tx back and closes it.
+        /// </summary>
+        public void Rollback()
+        {
+            lock (this)
+            {
+                ThrowIfClosed();
+
+                _state = new StateHolder(_txs.TxRollback(this));
+            }
+        }
+
+        /// <summary>
+        /// Sets the rollback only flag.
+        /// </summary>
+        public bool SetRollbackOnly()
+        {
+            lock (this)
+            {
+                ThrowIfClosed();
+
+                return _txs.TxSetRollbackOnly(this);
+            }
+        }
+
+        /// <summary>
+        /// Gets a value indicating whether this instance is rollback only.
+        /// </summary>
+        public bool IsRollbackOnly
+        {
+            get
+            {
+                lock (this)
+                {
+                    var state0 = _state == null ? State : _state.State;
+
+                    return state0 == TransactionState.MarkedRollback ||
+                           state0 == TransactionState.RollingBack ||
+                           state0 == TransactionState.RolledBack;
+                }
+            }
+        }
+
+        /// <summary>
+        /// Gets the state.
+        /// </summary>
+        public TransactionState State
+        {
+            get
+            {
+                lock (this)
+                {
+                    return _state != null ? _state.State : _txs.TxState(this);
+                }
+            }
+        }
+
+        /// <summary>
+        /// Gets the isolation.
+        /// </summary>
+        public TransactionIsolation Isolation
+        {
+            get { return _isolation; }
+        }
+
+        /// <summary>
+        /// Gets the concurrency.
+        /// </summary>
+        public TransactionConcurrency Concurrency
+        {
+            get { return _concurrency; }
+        }
+
+        /// <summary>
+        /// Gets the timeout.
+        /// </summary>
+        public TimeSpan Timeout
+        {
+            get { return _timeout; }
+        }
+
+        /// <summary>
+        /// Gets the start time.
+        /// </summary>
+        public DateTime StartTime
+        {
+            get { return _startTime; }
+        }
+
+
+        /// <summary>
+        /// Gets the node identifier.
+        /// </summary>
+        public Guid NodeId
+        {
+            get { return _nodeId; }
+        }
+
+        /// <summary>
+        /// Gets the thread identifier.
+        /// </summary>
+        public long ThreadId
+        {
+            get { return _threadId; }
+        }
+
+        /// <summary>
+        /// Adds a new metadata.
+        /// </summary>
+        public void AddMeta<TV>(string name, TV val)
+        {
+            if (name == null)
+                throw new ArgumentException("Meta name cannot be null.");
+
+            lock (this)
+            {
+                if (_metas != null)
+                {
+                    int putIdx = -1;
+
+                    for (int i = 0; i < _metas.Length; i += 2)
+                    {
+                        if (name.Equals(_metas[i]))
+                        {
+                            _metas[i + 1] = val;
+
+                            return;
+                        }
+                        if (_metas[i] == null && putIdx == -1)
+                            // Preserve empty space index.
+                            putIdx = i;
+                    }
+
+                    // No meta with the given name found.
+                    if (putIdx == -1)
+                    {
+                        // Extend array.
+                        putIdx = _metas.Length;
+
+                        object[] metas0 = new object[putIdx + 2];
+
+                        Array.Copy(_metas, metas0, putIdx);
+
+                        _metas = metas0;
+                    }
+                    
+                    _metas[putIdx] = name;
+                    _metas[putIdx + 1] = val;
+                }
+                else
+                    _metas = new object[] { name, val };
+            }
+        }
+
+        /// <summary>
+        /// Gets metadata by name.
+        /// </summary>
+        public TV Meta<TV>(string name)
+        {
+            if (name == null)
+                throw new ArgumentException("Meta name cannot be null.");
+
+            lock (this)
+            {
+                if (_metas != null)
+                {
+                    for (int i = 0; i < _metas.Length; i += 2)
+                    {
+                        if (name.Equals(_metas[i]))
+                            return (TV)_metas[i + 1];
+                    }
+                }
+
+                return default(TV);
+            }
+        }
+
+        /// <summary>
+        /// Removes metadata by name.
+        /// </summary>
+        public TV RemoveMeta<TV>(string name)
+        {
+            if (name == null)
+                throw new ArgumentException("Meta name cannot be null.");
+
+            lock (this)
+            {
+                if (_metas != null)
+                {
+                    for (int i = 0; i < _metas.Length; i += 2)
+                    {
+                        if (name.Equals(_metas[i]))
+                        {
+                            TV val = (TV)_metas[i + 1];
+
+                            _metas[i] = null;
+                            _metas[i + 1] = null;
+
+                            return val;
+                        }
+                    }
+                }
+
+                return default(TV);
+            }
+        }
+
+        /// <summary>
+        /// Commits tx in async mode.
+        /// </summary>
+        internal IFuture CommitAsync()
+        {
+            lock (this)
+            {
+                ThrowIfClosed();
+
+                var fut = _txs.CommitAsync(this);
+
+                CloseWhenComplete(fut);
+
+                return fut;
+            }
+        }
+
+        /// <summary>
+        /// Rolls tx back in async mode.
+        /// </summary>
+        internal IFuture RollbackAsync()
+        {
+            lock (this)
+            {
+                ThrowIfClosed();
+
+                var fut = _txs.RollbackAsync(this);
+
+                CloseWhenComplete(fut);
+
+                return fut;
+            }
+        }
+
+        /// <summary>
+        /// Transaction ID.
+        /// </summary>
+        internal long Id
+        {
+            get { return _id; }
+        }
+
+        /** <inheritdoc /> */
+        public void Dispose()
+        {
+            try
+            {
+                Close();
+            }
+            finally
+            {
+                GC.SuppressFinalize(this);
+            }
+        }
+
+        /// <summary>
+        /// Gets a value indicating whether this transaction is closed.
+        /// </summary>
+        internal bool IsClosed
+        {
+            get { return _state != null; }
+        }
+
+        /// <summary>
+        /// Gets the closed exception.
+        /// </summary>
+        private InvalidOperationException GetClosedException()
+        {
+            return new InvalidOperationException(string.Format("Transaction {0} is closed, state is {1}", Id, State));
+        }
+
+        /// <summary>
+        /// Creates a future via provided factory if IsClosed is false; otherwise, return a future with an error.
+        /// </summary>
+        internal IFuture GetFutureOrError(Func<IFuture> operationFactory)
+        {
+            lock (this)
+            {
+                return IsClosed ? GetExceptionFuture() : operationFactory();
+            }
+        }
+
+        /// <summary>
+        /// Gets the future that throws an exception.
+        /// </summary>
+        private IFuture GetExceptionFuture()
+        {
+            var fut = new Future<object>();
+
+            fut.OnError(GetClosedException());
+
+            return fut;
+        }
+
+        /// <summary>
+        /// Closes the transaction and releases unmanaged resources.
+        /// </summary>
+        private void Close()
+        {
+            lock (this)
+            {
+                _state = _state ?? new StateHolder((TransactionState) _txs.TxClose(this));
+            }
+        }
+
+        /// <summary>
+        /// Throws and exception if transaction is closed.
+        /// </summary>
+        private void ThrowIfClosed()
+        {
+            if (IsClosed)
+                throw GetClosedException();
+        }
+
+        /// <summary>
+        /// Closes this transaction upon future completion.
+        /// </summary>
+        private void CloseWhenComplete(IFuture fut)
+        {
+            fut.Listen(Close);
+        }
+
+        /** <inheritdoc /> */
+        ~TransactionImpl()
+        {
+            Dispose();
+        }
+
+        /// <summary>
+        /// State holder.
+        /// </summary>
+        private class StateHolder
+        {
+            /** Current state. */
+            private readonly TransactionState _state;
+
+            /// <summary>
+            /// Constructor.
+            /// </summary>
+            /// <param name="state">State.</param>
+            public StateHolder(TransactionState state)
+            {
+                _state = state;
+            }
+
+            /// <summary>
+            /// Current state.
+            /// </summary>
+            public TransactionState State
+            {
+                get { return _state; }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionMetricsImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionMetricsImpl.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionMetricsImpl.cs
new file mode 100644
index 0000000..e2528f4
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionMetricsImpl.cs
@@ -0,0 +1,62 @@
+/*
+ * 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.Transactions
+{
+    using System;
+    using Apache.Ignite.Core.Portable;
+    using Apache.Ignite.Core.Transactions;
+
+    /// <summary>
+    /// Transaction metrics.
+    /// </summary>
+    internal class TransactionMetricsImpl : ITransactionMetrics
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="TransactionMetricsImpl"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        public TransactionMetricsImpl(IPortableRawReader reader)
+        {
+            CommitTime = reader.ReadDate() ?? default(DateTime);
+            RollbackTime = reader.ReadDate() ?? default(DateTime);
+
+            TxCommits = reader.ReadInt();
+            TxRollbacks = reader.ReadInt();
+        }
+
+        /// <summary>
+        /// Gets the last time transaction was committed.
+        /// </summary>
+        public DateTime CommitTime { get; private set; }
+
+        /// <summary>
+        /// Gets the last time transaction was rolled back.
+        /// </summary>
+        public DateTime RollbackTime { get; private set; }
+
+        /// <summary>
+        /// Gets the total number of transaction commits.
+        /// </summary>
+        public int TxCommits { get; private set; }
+
+        /// <summary>
+        /// Gets the total number of transaction rollbacks.
+        /// </summary>
+        public int TxRollbacks { get; private set; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs
new file mode 100644
index 0000000..4eaa53f
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs
@@ -0,0 +1,201 @@
+/*
+ * 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.Transactions
+{
+    using System;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+    using Apache.Ignite.Core.Portable;
+    using Apache.Ignite.Core.Transactions;
+    using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils;
+
+    /// <summary>
+    /// Transactions facade.
+    /// </summary>
+    internal class TransactionsImpl : PlatformTarget, ITransactions
+    {
+        /** */
+        private const int OpCacheConfigParameters = 1;
+
+        /** */
+        private const int OpMetrics = 2;
+        
+        /** */
+        private readonly TransactionConcurrency _dfltConcurrency;
+
+        /** */
+        private readonly TransactionIsolation _dfltIsolation;
+
+        /** */
+        private readonly TimeSpan _dfltTimeout;
+
+        /** */
+        private readonly Guid _localNodeId;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="TransactionsImpl" /> class.
+        /// </summary>
+        /// <param name="target">Target.</param>
+        /// <param name="marsh">Marshaller.</param>
+        /// <param name="localNodeId">Local node id.</param>
+        public TransactionsImpl(IUnmanagedTarget target, PortableMarshaller marsh,
+            Guid localNodeId) : base(target, marsh)
+        {
+            _localNodeId = localNodeId;
+
+            TransactionConcurrency concurrency = default(TransactionConcurrency);
+            TransactionIsolation isolation = default(TransactionIsolation);
+            TimeSpan timeout = default(TimeSpan);
+
+            DoInOp(OpCacheConfigParameters, stream =>
+            {
+                var reader = marsh.StartUnmarshal(stream).RawReader();
+
+                concurrency = reader.ReadEnum<TransactionConcurrency>();
+                isolation = reader.ReadEnum<TransactionIsolation>();
+                timeout = TimeSpan.FromMilliseconds(reader.ReadLong());
+            });
+
+            _dfltConcurrency = concurrency;
+            _dfltIsolation = isolation;
+            _dfltTimeout = timeout;
+        }
+
+        /** <inheritDoc /> */
+        public ITransaction TxStart()
+        {
+            return TxStart(_dfltConcurrency, _dfltIsolation);
+        }
+
+        /** <inheritDoc /> */
+        public ITransaction TxStart(TransactionConcurrency concurrency, TransactionIsolation isolation)
+        {
+            return TxStart(concurrency, isolation, _dfltTimeout, 0);
+        }
+
+        /** <inheritDoc /> */
+        public ITransaction TxStart(TransactionConcurrency concurrency, TransactionIsolation isolation,
+            TimeSpan timeout, int txSize)
+        {
+            var id = UU.TransactionsStart(Target, (int)concurrency, (int)isolation, (long)timeout.TotalMilliseconds,
+                txSize);
+
+            var innerTx = new TransactionImpl(id, this, concurrency, isolation, timeout, _localNodeId);
+            
+            return new Transaction(innerTx);
+        }
+
+        /** <inheritDoc /> */
+        public ITransaction Tx
+        {
+            get { return TransactionImpl.Current; }
+        }
+
+        /** <inheritDoc /> */
+        public ITransactionMetrics GetMetrics()
+        {
+            return DoInOp(OpMetrics, stream =>
+            {
+                IPortableRawReader reader = Marshaller.StartUnmarshal(stream, false);
+
+                return new TransactionMetricsImpl(reader);
+            });
+        }
+
+        /** <inheritDoc /> */
+        public void ResetMetrics()
+        {
+            UU.TransactionsResetMetrics(Target);
+        }
+
+        /// <summary>
+        /// Commit transaction.
+        /// </summary>
+        /// <param name="tx">Transaction.</param>
+        /// <returns>Final transaction state.</returns>
+        internal TransactionState TxCommit(TransactionImpl tx)
+        {
+            return (TransactionState) UU.TransactionsCommit(Target, tx.Id);
+        }
+
+        /// <summary>
+        /// Rollback transaction.
+        /// </summary>
+        /// <param name="tx">Transaction.</param>
+        /// <returns>Final transaction state.</returns>
+        internal TransactionState TxRollback(TransactionImpl tx)
+        {
+            return (TransactionState)UU.TransactionsRollback(Target, tx.Id);
+        }
+
+        /// <summary>
+        /// Close transaction.
+        /// </summary>
+        /// <param name="tx">Transaction.</param>
+        /// <returns>Final transaction state.</returns>
+        internal int TxClose(TransactionImpl tx)
+        {
+            return UU.TransactionsClose(Target, tx.Id);
+        }
+
+        /// <summary>
+        /// Get transaction current state.
+        /// </summary>
+        /// <param name="tx">Transaction.</param>
+        /// <returns>Transaction current state.</returns>
+        internal TransactionState TxState(TransactionImpl tx)
+        {
+            return GetTransactionState(UU.TransactionsState(Target, tx.Id));
+        }
+
+        /// <summary>
+        /// Set transaction rollback-only flag.
+        /// </summary>
+        /// <param name="tx">Transaction.</param>
+        /// <returns><c>true</c> if the flag was set.</returns>
+        internal bool TxSetRollbackOnly(TransactionImpl tx)
+        {
+            return UU.TransactionsSetRollbackOnly(Target, tx.Id);
+        }
+
+        /// <summary>
+        /// Commits tx in async mode.
+        /// </summary>
+        internal IFuture CommitAsync(TransactionImpl tx)
+        {
+            return GetFuture<object>((futId, futTyp) => UU.TransactionsCommitAsync(Target, tx.Id, futId));
+        }
+
+        /// <summary>
+        /// Rolls tx back in async mode.
+        /// </summary>
+        internal IFuture RollbackAsync(TransactionImpl tx)
+        {
+            return GetFuture<object>((futId, futTyp) => UU.TransactionsRollbackAsync(Target, tx.Id, futId));
+        }
+ 
+        /// <summary>
+        /// Gets the state of the transaction from int.
+        /// </summary>
+        private static TransactionState GetTransactionState(int state)
+        {
+            return (TransactionState)state;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IUnmanagedTarget.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IUnmanagedTarget.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IUnmanagedTarget.cs
new file mode 100644
index 0000000..235f20d
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IUnmanagedTarget.cs
@@ -0,0 +1,42 @@
+/*
+ * 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.Unmanaged
+{
+    using System;
+
+    /// <summary>
+    /// Unmanaged target.
+    /// </summary>
+    internal unsafe interface IUnmanagedTarget : IDisposable
+    {
+        /// <summary>
+        /// Context.
+        /// </summary>
+        void* Context { get; }
+
+        /// <summary>
+        /// Target.
+        /// </summary>
+        void* Target { get; }
+
+        /// <summary>
+        /// Creates new instance with same context and different target.
+        /// </summary>
+        IUnmanagedTarget ChangeTarget(void* target);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackHandlers.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackHandlers.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackHandlers.cs
new file mode 100644
index 0000000..07cf309
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackHandlers.cs
@@ -0,0 +1,99 @@
+/*
+ * 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.Unmanaged
+{
+    using System.Runtime.InteropServices;
+
+    /// <summary>
+    /// Unmanaged callback handler function pointers.
+    /// </summary>
+    [StructLayout(LayoutKind.Sequential)]
+    internal unsafe struct UnmanagedCallbackHandlers
+    {
+        internal void* target;
+
+        internal void* cacheStoreCreate;
+        internal void* cacheStoreInvoke;
+        internal void* cacheStoreDestroy;
+        internal void* cacheStoreSessionCreate;
+
+        internal void* cacheEntryFilterCreate;
+        internal void* cacheEntryFilterApply;
+        internal void* cacheEntryFilterDestroy;
+
+        internal void* cacheInvoke;
+
+        internal void* computeTaskMap;
+        internal void* computeTaskJobResult;
+        internal void* computeTaskReduce;
+        internal void* computeTaskComplete;
+        internal void* computeJobSerialize;
+        internal void* computeJobCreate;
+        internal void* computeJobExecute;
+        internal void* computeJobCancel;
+        internal void* computeJobDestroy;
+
+        internal void* continuousQueryListenerApply;
+        internal void* continuousQueryFilterCreate;
+        internal void* continuousQueryFilterApply;
+        internal void* continuousQueryFilterRelease;
+
+        internal void* dataStreamerTopologyUpdate;
+        internal void* dataStreamerStreamReceiverInvoke;
+        
+        internal void* futureByteResult;
+        internal void* futureBoolResult;
+        internal void* futureShortResult;
+        internal void* futureCharResult;
+        internal void* futureIntResult;
+        internal void* futureFloatResult;
+        internal void* futureLongResult;
+        internal void* futureDoubleResult;
+        internal void* futureObjectResult;
+        internal void* futureNullResult;
+        internal void* futureError;
+
+        internal void* lifecycleOnEvent;
+
+        internal void* memoryReallocate;
+
+        internal void* messagingFilterCreate;
+        internal void* messagingFilterApply;
+        internal void* messagingFilterDestroy;
+        
+        internal void* eventFilterCreate;
+        internal void* eventFilterApply;
+        internal void* eventFilterDestroy;
+
+        internal void* serviceInit;
+        internal void* serviceExecute;
+        internal void* serviceCancel;
+        internal void* serviceInvokeMethod;
+
+        internal void* clusterNodeFilterApply;
+
+        internal void* nodeInfo;
+
+        internal void* onStart;
+        internal void* onStop;
+        internal void* error;
+
+        internal void* extensionCbInLongOutLong;
+        internal void* extensionCbInLongLongOutLong;
+    }
+}


[27/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Ignite.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
new file mode 100644
index 0000000..7e33416
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
@@ -0,0 +1,547 @@
+/*
+ * 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
+{
+    using System;
+    using System.Collections.Concurrent;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Datastream;
+    using Apache.Ignite.Core.Events;
+    using Apache.Ignite.Core.Impl.Cache;
+    using Apache.Ignite.Core.Impl.Cluster;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Datastream;
+    using Apache.Ignite.Core.Impl.Handle;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Transactions;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+    using Apache.Ignite.Core.Lifecycle;
+    using Apache.Ignite.Core.Messaging;
+    using Apache.Ignite.Core.Portable;
+    using Apache.Ignite.Core.Services;
+    using Apache.Ignite.Core.Transactions;
+    using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils;
+
+    /// <summary>
+    /// Native Ignite wrapper.
+    /// </summary>
+    internal class Ignite : IIgnite, IClusterGroupEx, ICluster
+    {
+        /** */
+        private readonly IgniteConfiguration _cfg;
+
+        /** Grid name. */
+        private readonly string _name;
+
+        /** Unmanaged node. */
+        private readonly IUnmanagedTarget _proc;
+
+        /** Marshaller. */
+        private readonly PortableMarshaller _marsh;
+
+        /** Initial projection. */
+        private readonly ClusterGroupImpl _prj;
+
+        /** Portables. */
+        private readonly PortablesImpl _portables;
+
+        /** Cached proxy. */
+        private readonly IgniteProxy _proxy;
+
+        /** Lifecycle beans. */
+        private readonly IList<LifecycleBeanHolder> _lifecycleBeans;
+
+        /** Local node. */
+        private IClusterNode _locNode;
+
+        /** Transactions facade. */
+        private readonly TransactionsImpl _transactions;
+
+        /** Callbacks */
+        private readonly UnmanagedCallbacks _cbs;
+
+        /** Node info cache. */
+
+        private readonly ConcurrentDictionary<Guid, ClusterNodeImpl> _nodes =
+            new ConcurrentDictionary<Guid, ClusterNodeImpl>();
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="cfg">Configuration.</param>
+        /// <param name="name">Grid name.</param>
+        /// <param name="proc">Interop processor.</param>
+        /// <param name="marsh">Marshaller.</param>
+        /// <param name="lifecycleBeans">Lifecycle beans.</param>
+        /// <param name="cbs">Callbacks.</param>
+        public Ignite(IgniteConfiguration cfg, string name, IUnmanagedTarget proc, PortableMarshaller marsh,
+            IList<LifecycleBeanHolder> lifecycleBeans, UnmanagedCallbacks cbs)
+        {
+            Debug.Assert(cfg != null);
+            Debug.Assert(proc != null);
+            Debug.Assert(marsh != null);
+            Debug.Assert(lifecycleBeans != null);
+            Debug.Assert(cbs != null);
+
+            _cfg = cfg;
+            _name = name;
+            _proc = proc;
+            _marsh = marsh;
+            _lifecycleBeans = lifecycleBeans;
+            _cbs = cbs;
+
+            marsh.Ignite = this;
+
+            _prj = new ClusterGroupImpl(proc, UU.ProcessorProjection(proc), marsh, this, null);
+
+            _portables = new PortablesImpl(marsh);
+
+            _proxy = new IgniteProxy(this);
+
+            cbs.Initialize(this);
+
+            _transactions = new TransactionsImpl(UU.ProcessorTransactions(proc), marsh, LocalNode.Id);
+        }
+
+        /// <summary>
+        /// On-start routine.
+        /// </summary>
+        internal void OnStart()
+        {
+            foreach (var lifecycleBean in _lifecycleBeans)
+                lifecycleBean.OnStart(this);
+        }
+
+        /// <summary>
+        /// Gets Ignite proxy.
+        /// </summary>
+        /// <returns>Proxy.</returns>
+        public IgniteProxy Proxy
+        {
+            get { return _proxy; }
+        }
+
+        /** <inheritdoc /> */
+        public string Name
+        {
+            get { return _name; }
+        }
+
+        /** <inheritdoc /> */
+        public ICluster Cluster
+        {
+            get { return this; }
+        }
+
+        /** <inheritdoc /> */
+        IIgnite IClusterGroup.Ignite
+        {
+            get { return this; }
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForLocal()
+        {
+            return _prj.ForNodes(LocalNode);
+        }
+
+        /** <inheritdoc /> */
+        public ICompute Compute()
+        {
+            return _prj.Compute();
+        }
+
+        /** <inheritdoc /> */
+        public ICompute Compute(IClusterGroup clusterGroup)
+        {
+            IgniteArgumentCheck.NotNull(clusterGroup, "clusterGroup");
+
+            return clusterGroup.Compute();
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForNodes(IEnumerable<IClusterNode> nodes)
+        {
+            return ((IClusterGroup) _prj).ForNodes(nodes);
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForNodes(params IClusterNode[] nodes)
+        {
+            return _prj.ForNodes(nodes);
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForNodeIds(IEnumerable<Guid> ids)
+        {
+            return ((IClusterGroup) _prj).ForNodeIds(ids);
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForNodeIds(ICollection<Guid> ids)
+        {
+            return _prj.ForNodeIds(ids);
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForNodeIds(params Guid[] ids)
+        {
+            return _prj.ForNodeIds(ids);
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForPredicate(Func<IClusterNode, bool> p)
+        {
+            IgniteArgumentCheck.NotNull(p, "p");
+
+            return _prj.ForPredicate(p);
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForAttribute(string name, string val)
+        {
+            return _prj.ForAttribute(name, val);
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForCacheNodes(string name)
+        {
+            return _prj.ForCacheNodes(name);
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForDataNodes(string name)
+        {
+            return _prj.ForDataNodes(name);
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForClientNodes(string name)
+        {
+            return _prj.ForClientNodes(name);
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForRemotes()
+        {
+            return _prj.ForRemotes();
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForHost(IClusterNode node)
+        {
+            IgniteArgumentCheck.NotNull(node, "node");
+
+            return _prj.ForHost(node);
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForRandom()
+        {
+            return _prj.ForRandom();
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForOldest()
+        {
+            return _prj.ForOldest();
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForYoungest()
+        {
+            return _prj.ForYoungest();
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForDotNet()
+        {
+            return _prj.ForDotNet();
+        }
+
+        /** <inheritdoc /> */
+        public ICollection<IClusterNode> Nodes()
+        {
+            return _prj.Nodes();
+        }
+
+        /** <inheritdoc /> */
+        public IClusterNode Node(Guid id)
+        {
+            return _prj.Node(id);
+        }
+
+        /** <inheritdoc /> */
+        public IClusterNode Node()
+        {
+            return _prj.Node();
+        }
+
+        /** <inheritdoc /> */
+        public IClusterMetrics Metrics()
+        {
+            return _prj.Metrics();
+        }
+
+        /** <inheritdoc /> */
+        public void Dispose()
+        {
+            Ignition.Stop(Name, true);
+        }
+
+        /// <summary>
+        /// Internal stop routine.
+        /// </summary>
+        /// <param name="cancel">Cancel flag.</param>
+        internal unsafe void Stop(bool cancel)
+        {
+            UU.IgnitionStop(_proc.Context, Name, cancel);
+
+            _cbs.Cleanup();
+
+            foreach (var bean in _lifecycleBeans)
+                bean.OnLifecycleEvent(LifecycleEventType.AfterNodeStop);
+        }
+
+        /** <inheritdoc /> */
+        public ICache<TK, TV> Cache<TK, TV>(string name)
+        {
+            return Cache<TK, TV>(UU.ProcessorCache(_proc, name));
+        }
+
+        /** <inheritdoc /> */
+        public ICache<TK, TV> GetOrCreateCache<TK, TV>(string name)
+        {
+            return Cache<TK, TV>(UU.ProcessorGetOrCreateCache(_proc, name));
+        }
+
+        /** <inheritdoc /> */
+        public ICache<TK, TV> CreateCache<TK, TV>(string name)
+        {
+            return Cache<TK, TV>(UU.ProcessorCreateCache(_proc, name));
+        }
+
+        /// <summary>
+        /// Gets cache from specified native cache object.
+        /// </summary>
+        /// <param name="nativeCache">Native cache.</param>
+        /// <param name="keepPortable">Portable flag.</param>
+        /// <returns>
+        /// New instance of cache wrapping specified native cache.
+        /// </returns>
+        public ICache<TK, TV> Cache<TK, TV>(IUnmanagedTarget nativeCache, bool keepPortable = false)
+        {
+            var cacheImpl = new CacheImpl<TK, TV>(this, nativeCache, _marsh, false, keepPortable, false, false);
+
+            return new CacheProxyImpl<TK, TV>(cacheImpl);
+        }
+
+        /** <inheritdoc /> */
+        public IClusterNode LocalNode
+        {
+            get
+            {
+                if (_locNode == null)
+                {
+                    foreach (IClusterNode node in Nodes())
+                    {
+                        if (node.IsLocal)
+                        {
+                            _locNode = node;
+
+                            break;
+                        }
+                    }
+                }
+
+                return _locNode;
+            }
+        }
+
+        /** <inheritdoc /> */
+        public bool PingNode(Guid nodeId)
+        {
+            return _prj.PingNode(nodeId);
+        }
+
+        /** <inheritdoc /> */
+        public long TopologyVersion
+        {
+            get { return _prj.TopologyVersion; }
+        }
+
+        /** <inheritdoc /> */
+        public ICollection<IClusterNode> Topology(long ver)
+        {
+            return _prj.Topology(ver);
+        }
+
+        /** <inheritdoc /> */
+        public void ResetMetrics()
+        {
+            UU.ProjectionResetMetrics(_prj.Target);
+        }
+
+        /** <inheritdoc /> */
+        public IDataStreamer<TK, TV> DataStreamer<TK, TV>(string cacheName)
+        {
+            return new DataStreamerImpl<TK, TV>(UU.ProcessorDataStreamer(_proc, cacheName, false),
+                _marsh, cacheName, false);
+        }
+
+        /** <inheritdoc /> */
+        public IPortables Portables()
+        {
+            return _portables;
+        }
+
+        /** <inheritdoc /> */
+        public ICacheAffinity Affinity(string cacheName)
+        {
+            return new CacheAffinityImpl(UU.ProcessorAffinity(_proc, cacheName), _marsh, false, this);
+        }
+
+        /** <inheritdoc /> */
+        public ITransactions Transactions
+        {
+            get { return _transactions; }
+        }
+
+        /** <inheritdoc /> */
+        public IMessaging Message()
+        {
+            return _prj.Message();
+        }
+
+        /** <inheritdoc /> */
+        public IMessaging Message(IClusterGroup clusterGroup)
+        {
+            IgniteArgumentCheck.NotNull(clusterGroup, "clusterGroup");
+
+            return clusterGroup.Message();
+        }
+
+        /** <inheritdoc /> */
+        public IEvents Events()
+        {
+            return _prj.Events();
+        }
+
+        /** <inheritdoc /> */
+        public IEvents Events(IClusterGroup clusterGroup)
+        {
+            if (clusterGroup == null)
+                throw new ArgumentNullException("clusterGroup");
+
+            return clusterGroup.Events();
+        }
+
+        /** <inheritdoc /> */
+        public IServices Services()
+        {
+            return _prj.Services();
+        }
+
+        /// <summary>
+        /// Gets internal projection.
+        /// </summary>
+        /// <returns>Projection.</returns>
+        internal ClusterGroupImpl ClusterGroup
+        {
+            get { return _prj; }
+        }
+
+        /// <summary>
+        /// Marshaller.
+        /// </summary>
+        internal PortableMarshaller Marshaller
+        {
+            get { return _marsh; }
+        }
+
+        /// <summary>
+        /// Configuration.
+        /// </summary>
+        internal IgniteConfiguration Configuration
+        {
+            get { return _cfg; }
+        }
+
+        /// <summary>
+        /// Put metadata to Grid.
+        /// </summary>
+        /// <param name="metas">Metadata.</param>
+        internal void PutMetadata(IDictionary<int, IPortableMetadata> metas)
+        {
+            _prj.PutMetadata(metas);
+        }
+
+        /** <inheritDoc /> */
+        public IPortableMetadata Metadata(int typeId)
+        {
+            return _prj.Metadata(typeId);
+        }
+
+        /// <summary>
+        /// Handle registry.
+        /// </summary>
+        public HandleRegistry HandleRegistry
+        {
+            get { return _cbs.HandleRegistry; }
+        }
+
+        /// <summary>
+        /// Updates the node information from stream.
+        /// </summary>
+        /// <param name="memPtr">Stream ptr.</param>
+        public void UpdateNodeInfo(long memPtr)
+        {
+            var stream = IgniteManager.Memory.Get(memPtr).Stream();
+
+            IPortableRawReader reader = Marshaller.StartUnmarshal(stream, false);
+
+            var node = new ClusterNodeImpl(reader);
+
+            node.Init(this);
+
+            _nodes[node.Id] = node;
+        }
+
+        /// <summary>
+        /// Gets the node from cache.
+        /// </summary>
+        /// <param name="id">Node id.</param>
+        /// <returns>Cached node.</returns>
+        public ClusterNodeImpl GetNode(Guid? id)
+        {
+            return id == null ? null : _nodes[id.Value];
+        }
+
+        /// <summary>
+        /// Gets the interop processor.
+        /// </summary>
+        internal IUnmanagedTarget InteropProcessor
+        {
+            get { return _proc; }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/IgniteConfigurationEx.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/IgniteConfigurationEx.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/IgniteConfigurationEx.cs
new file mode 100644
index 0000000..358e805
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/IgniteConfigurationEx.cs
@@ -0,0 +1,57 @@
+/*
+ * 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
+{
+    /// <summary>
+    /// Internal extensions for IgniteConfiguration.
+    /// </summary>
+    internal class IgniteConfigurationEx : IgniteConfiguration
+    {
+        /// <summary>
+        /// Default constructor.
+        /// </summary>
+        public IgniteConfigurationEx()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Copying constructor.
+        /// </summary>
+        /// <param name="cfg">Configuration.</param>
+        public IgniteConfigurationEx(IgniteConfiguration cfg) : base(cfg)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Copying constructor.
+        /// </summary>
+        /// <param name="cfg">Configuration.</param>
+        public IgniteConfigurationEx(IgniteConfigurationEx cfg)
+            : this((IgniteConfiguration) cfg)
+        {
+            GridName = cfg.GridName;
+        }
+
+        /// <summary>
+        /// Grid name which is used if not provided in configuration file.
+        /// </summary>
+        public string GridName { get; set; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/IgniteManager.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/IgniteManager.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/IgniteManager.cs
new file mode 100644
index 0000000..6203b3c
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/IgniteManager.cs
@@ -0,0 +1,492 @@
+/*
+ * 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
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Diagnostics.CodeAnalysis;
+    using System.IO;
+    using System.Linq;
+    using System.Reflection;
+    using System.Runtime.InteropServices;
+    using System.Text;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Memory;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+    using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils;
+
+    /// <summary>
+    /// Native interface manager.
+    /// </summary>
+    internal static unsafe class IgniteManager
+    {
+        /** Environment variable: IGNITE_HOME. */
+        internal const string EnvIgniteHome = "IGNITE_HOME";
+
+        /** Environment variable: whether to set test classpath or not. */
+        private const string EnvIgniteNativeTestClasspath = "IGNITE_NATIVE_TEST_CLASSPATH";
+        
+        /** Classpath prefix. */
+        private const string ClasspathPrefix = "-Djava.class.path=";
+
+        /** Java Command line argument: Xms. Case sensitive. */
+        private const string CmdJvmMinMemJava = "-Xms";
+
+        /** Java Command line argument: Xmx. Case sensitive. */
+        private const string CmdJvmMaxMemJava = "-Xmx";
+
+        /** Monitor for DLL load synchronization. */
+        private static readonly object SyncRoot = new object();
+
+        /** First created context. */
+        private static void* _ctx;
+
+        /** Configuration used on JVM start. */
+        private static JvmConfiguration _jvmCfg;
+
+        /** Memory manager. */
+        private static PlatformMemoryManager _mem;
+
+        /// <summary>
+        /// Static initializer.
+        /// </summary>
+        static IgniteManager()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Create JVM.
+        /// </summary>
+        /// <param name="cfg">Configuration.</param>
+        /// <param name="cbs">Callbacks.</param>
+        /// <returns>Context.</returns>
+        internal static void* GetContext(IgniteConfiguration cfg, UnmanagedCallbacks cbs)
+        {
+            lock (SyncRoot)
+            {
+                // 1. Warn about possible configuration inconsistency.
+                JvmConfiguration jvmCfg = JvmConfig(cfg);
+
+                if (!cfg.SuppressWarnings && _jvmCfg != null)
+                {
+                    if (!_jvmCfg.Equals(jvmCfg))
+                    {
+                        Console.WriteLine("Attempting to start Ignite node with different Java " +
+                            "configuration; current Java configuration will be ignored (consider " +
+                            "starting node in separate process) [oldConfig=" + _jvmCfg +
+                            ", newConfig=" + jvmCfg + ']');
+                    }
+                }
+
+                // 2. Create unmanaged pointer.
+                void* ctx = CreateJvm(cfg, cbs);
+
+                cbs.SetContext(ctx);
+
+                // 3. If this is the first JVM created, preserve it.
+                if (_ctx == null)
+                {
+                    _ctx = ctx;
+                    _jvmCfg = jvmCfg;
+                    _mem = new PlatformMemoryManager(1024);
+                }
+
+                return ctx;
+            }
+        }
+        
+        /// <summary>
+        /// Memory manager attached to currently running JVM.
+        /// </summary>
+        internal static PlatformMemoryManager Memory
+        {
+            get { return _mem; }
+        }
+
+        /// <summary>
+        /// Destroy JVM.
+        /// </summary>
+        public static void DestroyJvm()
+        {
+            lock (SyncRoot)
+            {
+                if (_ctx != null)
+                {
+                    UU.DestroyJvm(_ctx);
+
+                    _ctx = null;
+                }
+            }
+        }
+
+        /// <summary>
+        /// Create JVM.
+        /// </summary>
+        /// <returns>JVM.</returns>
+        private static void* CreateJvm(IgniteConfiguration cfg, UnmanagedCallbacks cbs)
+        {
+            var ggHome = GetIgniteHome(cfg);
+
+            var cp = CreateClasspath(ggHome, cfg, false);
+
+            var jvmOpts = GetMergedJvmOptions(cfg);
+            
+            var hasGgHome = !string.IsNullOrWhiteSpace(ggHome);
+
+            var opts = new sbyte*[1 + jvmOpts.Count + (hasGgHome ? 1 : 0)];
+
+            int idx = 0;
+                
+            opts[idx++] = IgniteUtils.StringToUtf8Unmanaged(cp);
+
+            if (hasGgHome)
+                opts[idx++] = IgniteUtils.StringToUtf8Unmanaged("-DIGNITE_HOME=" + ggHome);
+
+            foreach (string cfgOpt in jvmOpts)
+                opts[idx++] = IgniteUtils.StringToUtf8Unmanaged(cfgOpt);
+
+            try
+            {
+                IntPtr mem = Marshal.AllocHGlobal(opts.Length * 8);
+
+                fixed (sbyte** opts0 = opts)
+                {
+                    PlatformMemoryUtils.CopyMemory(opts0, mem.ToPointer(), opts.Length * 8);
+                }
+
+                try
+                {
+                    return UU.CreateContext(mem.ToPointer(), opts.Length, cbs.CallbacksPointer);
+                }
+                finally
+                {
+                    Marshal.FreeHGlobal(mem);
+                }
+            }
+            finally
+            {
+                foreach (sbyte* opt in opts)
+                    Marshal.FreeHGlobal((IntPtr)opt);
+            }
+        }
+
+        /// <summary>
+        /// Gets JvmOptions collection merged with individual properties (Min/Max mem, etc) according to priority.
+        /// </summary>
+        private static IList<string> GetMergedJvmOptions(IgniteConfiguration cfg)
+        {
+            var jvmOpts = cfg.JvmOptions == null ? new List<string>() : cfg.JvmOptions.ToList();
+
+            // JvmInitialMemoryMB / JvmMaxMemoryMB have lower priority than CMD_JVM_OPT
+            if (!jvmOpts.Any(opt => opt.StartsWith(CmdJvmMinMemJava, StringComparison.OrdinalIgnoreCase)))
+                jvmOpts.Add(string.Format("{0}{1}m", CmdJvmMinMemJava, cfg.JvmInitialMemoryMb));
+
+            if (!jvmOpts.Any(opt => opt.StartsWith(CmdJvmMaxMemJava, StringComparison.OrdinalIgnoreCase)))
+                jvmOpts.Add(string.Format("{0}{1}m", CmdJvmMaxMemJava, cfg.JvmMaxMemoryMb));
+
+            return jvmOpts;
+        }
+
+        /// <summary>
+        /// Create JVM configuration value object.
+        /// </summary>
+        /// <param name="cfg">Configuration.</param>
+        /// <returns>JVM configuration.</returns>
+        private static JvmConfiguration JvmConfig(IgniteConfiguration cfg)
+        {
+            return new JvmConfiguration
+            {
+                Home = cfg.IgniteHome,
+                Dll = cfg.JvmDllPath,
+                Classpath = cfg.JvmClasspath,
+                Options = cfg.JvmOptions
+            };
+        }
+
+        /// <summary>
+        /// Append jars from the given path.
+        /// </summary>
+        /// <param name="path">Path.</param>
+        /// <param name="cpStr">Classpath string builder.</param>
+        private static void AppendJars(string path, StringBuilder cpStr)
+        {
+            if (Directory.Exists(path))
+            {
+                foreach (string jar in Directory.EnumerateFiles(path, "*.jar"))
+                {
+                    cpStr.Append(jar);
+                    cpStr.Append(';');
+                }
+            }
+        }
+
+        /// <summary>
+        /// Calculate Ignite home.
+        /// </summary>
+        /// <param name="cfg">Configuration.</param>
+        /// <returns></returns>
+        internal static string GetIgniteHome(IgniteConfiguration cfg)
+        {
+            var home = cfg == null ? null : cfg.IgniteHome;
+
+            if (string.IsNullOrWhiteSpace(home))
+                home = Environment.GetEnvironmentVariable(EnvIgniteHome);
+            else if (!IsIgniteHome(new DirectoryInfo(home)))
+                throw new IgniteException(string.Format("IgniteConfiguration.IgniteHome is not valid: '{0}'", home));
+
+            if (string.IsNullOrWhiteSpace(home))
+                home = ResolveIgniteHome();
+            else if (!IsIgniteHome(new DirectoryInfo(home)))
+                throw new IgniteException(string.Format("{0} is not valid: '{1}'", EnvIgniteHome, home));
+
+            return home;
+        }
+
+        /// <summary>
+        /// Automatically resolve Ignite home directory.
+        /// </summary>
+        /// <returns>Ignite home directory.</returns>
+        private static string ResolveIgniteHome()
+        {
+            var probeDirs = new[]
+            {
+                Path.GetDirectoryName(Assembly.GetExecutingAssembly().Location),
+                Directory.GetCurrentDirectory()
+            };
+
+            foreach (var probeDir in probeDirs.Where(x => !string.IsNullOrEmpty(x)))
+            {
+                var dir = new DirectoryInfo(probeDir);
+
+                while (dir != null)
+                {
+                    if (IsIgniteHome(dir))
+                        return dir.FullName;
+
+                    dir = dir.Parent;
+                }
+            }
+
+            return null;
+        }
+
+        /// <summary>
+        /// Determines whether specified dir looks like a Ignite home.
+        /// </summary>
+        /// <param name="dir">Directory.</param>
+        /// <returns>Value indicating whether specified dir looks like a Ignite home.</returns>
+        private static bool IsIgniteHome(DirectoryInfo dir)
+        {
+            return dir.Exists && dir.EnumerateDirectories().Count(x => x.Name == "examples" || x.Name == "bin") == 2;
+        }
+
+        /// <summary>
+        /// Creates classpath from the given configuration, or default classpath if given config is null.
+        /// </summary>
+        /// <param name="cfg">The configuration.</param>
+        /// <param name="forceTestClasspath">Append test directories even if <see cref="EnvIgniteNativeTestClasspath" /> is not set.</param>
+        /// <returns>
+        /// Classpath string.
+        /// </returns>
+        internal static string CreateClasspath(IgniteConfiguration cfg = null, bool forceTestClasspath = false)
+        {
+            return CreateClasspath(GetIgniteHome(cfg), cfg, forceTestClasspath);
+        }
+
+        /// <summary>
+        /// Creates classpath from the given configuration, or default classpath if given config is null.
+        /// </summary>
+        /// <param name="ggHome">The home dir.</param>
+        /// <param name="cfg">The configuration.</param>
+        /// <param name="forceTestClasspath">Append test directories even if
+        ///     <see cref="EnvIgniteNativeTestClasspath" /> is not set.</param>
+        /// <returns>
+        /// Classpath string.
+        /// </returns>
+        private static string CreateClasspath(string ggHome, IgniteConfiguration cfg, bool forceTestClasspath)
+        {
+            var cpStr = new StringBuilder();
+
+            if (cfg != null && cfg.JvmClasspath != null)
+            {
+                cpStr.Append(cfg.JvmClasspath);
+
+                if (!cfg.JvmClasspath.EndsWith(";"))
+                    cpStr.Append(';');
+            }
+
+            if (!string.IsNullOrWhiteSpace(ggHome))
+                AppendHomeClasspath(ggHome, forceTestClasspath, cpStr);
+
+            return ClasspathPrefix + cpStr;
+        }
+
+        /// <summary>
+        /// Appends classpath from home directory, if it is defined.
+        /// </summary>
+        /// <param name="ggHome">The home dir.</param>
+        /// <param name="forceTestClasspath">Append test directories even if
+        ///     <see cref="EnvIgniteNativeTestClasspath"/> is not set.</param>
+        /// <param name="cpStr">The classpath string.</param>
+        private static void AppendHomeClasspath(string ggHome, bool forceTestClasspath, StringBuilder cpStr)
+        {
+            // Append test directories (if needed) first, because otherwise build *.jar will be picked first.
+            if (forceTestClasspath || "true".Equals(Environment.GetEnvironmentVariable(EnvIgniteNativeTestClasspath)))
+            {
+                AppendTestClasses(ggHome + "\\examples", cpStr);
+                AppendTestClasses(ggHome + "\\modules", cpStr);
+                AppendTestClasses(ggHome + "\\..\\incubator-ignite\\examples", cpStr);
+                AppendTestClasses(ggHome + "\\..\\incubator-ignite\\modules", cpStr);
+            }
+
+            string ggLibs = ggHome + "\\libs";
+
+            AppendJars(ggLibs, cpStr);
+
+            if (Directory.Exists(ggLibs))
+            {
+                foreach (string dir in Directory.EnumerateDirectories(ggLibs))
+                {
+                    if (!dir.EndsWith("optional"))
+                        AppendJars(dir, cpStr);
+                }
+            }
+        }
+
+        /// <summary>
+        /// Append target (compile) directories to classpath (for testing purposes only).
+        /// </summary>
+        /// <param name="path">Path</param>
+        /// <param name="cp">Classpath builder.</param>
+        private static void AppendTestClasses(string path, StringBuilder cp)
+        {
+            if (Directory.Exists(path))
+            {
+                AppendTestClasses0(path, cp);
+
+                foreach (string moduleDir in Directory.EnumerateDirectories(path))
+                    AppendTestClasses0(moduleDir, cp);
+            }
+        }
+
+        /// <summary>
+        /// Internal routine to append classes and jars from eploded directory.
+        /// </summary>
+        /// <param name="path">Path.</param>
+        /// <param name="cp">Classpath builder.</param>
+        private static void AppendTestClasses0(string path, StringBuilder cp)
+        {
+            if (path.EndsWith("rest-http", StringComparison.OrdinalIgnoreCase))
+                return;
+            
+            if (Directory.Exists(path + "\\target\\classes"))
+                cp.Append(path + "\\target\\classes;");
+
+            if (Directory.Exists(path + "\\target\\test-classes"))
+                cp.Append(path + "\\target\\test-classes;");
+
+            if (Directory.Exists(path + "\\target\\libs"))
+                AppendJars(path + "\\target\\libs", cp);
+        }
+
+        /// <summary>
+        /// JVM configuration.
+        /// </summary>
+        private class JvmConfiguration
+        {
+            /// <summary>
+            /// Gets or sets the home.
+            /// </summary>
+            public string Home { get; set; }
+
+            /// <summary>
+            /// Gets or sets the DLL.
+            /// </summary>
+            public string Dll { get; set; }
+
+            /// <summary>
+            /// Gets or sets the cp.
+            /// </summary>
+            public string Classpath { get; set; }
+
+            /// <summary>
+            /// Gets or sets the options.
+            /// </summary>
+            public ICollection<string> Options { get; set; }
+
+            /** <inheritDoc /> */
+            public override int GetHashCode()
+            {
+                return 0;
+            }
+
+            /** <inheritDoc /> */
+            [SuppressMessage("ReSharper", "FunctionComplexityOverflow")]
+            public override bool Equals(object obj)
+            {
+                JvmConfiguration other = obj as JvmConfiguration;
+
+                if (other == null)
+                    return false;
+
+                if (!string.Equals(Home, other.Home, StringComparison.OrdinalIgnoreCase))
+                    return false;
+
+                if (!string.Equals(Classpath, other.Classpath, StringComparison.OrdinalIgnoreCase))
+                    return false;
+
+                if (!string.Equals(Dll, other.Dll, StringComparison.OrdinalIgnoreCase))
+                    return false;
+
+                return (Options == null && other.Options == null) ||
+                       (Options != null && other.Options != null && Options.Count == other.Options.Count
+                        && !Options.Except(other.Options).Any());
+            }
+
+            /** <inheritDoc /> */
+            public override string ToString()
+            {
+                var sb = new StringBuilder("[IgniteHome=" + Home + ", JvmDllPath=" + Dll);
+
+                if (Options != null && Options.Count > 0)
+                {
+                    sb.Append(", JvmOptions=[");
+
+                    bool first = true;
+
+                    foreach (string opt in Options)
+                    {
+                        if (first)
+                            first = false;
+                        else
+                            sb.Append(", ");
+
+                        sb.Append(opt);
+                    }
+
+                    sb.Append(']');
+                }
+
+                sb.Append(", Classpath=" + Classpath + ']');
+
+                return sb.ToString();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs
new file mode 100644
index 0000000..f180830
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs
@@ -0,0 +1,351 @@
+/*
+ * 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
+{
+    using System;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Datastream;
+    using Apache.Ignite.Core.Events;
+    using Apache.Ignite.Core.Impl.Cluster;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Messaging;
+    using Apache.Ignite.Core.Portable;
+    using Apache.Ignite.Core.Services;
+    using Apache.Ignite.Core.Transactions;
+
+    /// <summary>
+    /// Grid proxy with fake serialization.
+    /// </summary>
+    [Serializable]
+    internal class IgniteProxy : IIgnite, IClusterGroupEx, IPortableWriteAware, ICluster
+    {
+        /** */
+        [NonSerialized]
+        private readonly IIgnite _ignite;
+
+        /// <summary>
+        /// Default ctor for marshalling.
+        /// </summary>
+        public IgniteProxy()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="ignite">Grid.</param>
+        public IgniteProxy(IIgnite ignite)
+        {
+            _ignite = ignite;
+        }
+
+        /** <inheritdoc /> */
+        public string Name
+        {
+            get { return _ignite.Name; }
+        }
+
+        /** <inheritdoc /> */
+        public ICluster Cluster
+        {
+            get { return this; }
+        }
+
+        /** <inheritdoc /> */
+        public IIgnite Ignite
+        {
+            get { return this; }
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForLocal()
+        {
+            return _ignite.Cluster.ForLocal();
+        }
+
+        /** <inheritdoc /> */
+        public ICompute Compute()
+        {
+            return _ignite.Compute();
+        }
+
+        /** <inheritdoc /> */
+        public ICompute Compute(IClusterGroup clusterGroup)
+        {
+            return clusterGroup.Compute();
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForNodes(IEnumerable<IClusterNode> nodes)
+        {
+            return _ignite.Cluster.ForNodes(nodes);
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForNodes(params IClusterNode[] nodes)
+        {
+            return _ignite.Cluster.ForNodes(nodes);
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForNodeIds(IEnumerable<Guid> ids)
+        {
+            return _ignite.Cluster.ForNodeIds(ids);
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForNodeIds(ICollection<Guid> ids)
+        {
+            return _ignite.Cluster.ForNodeIds(ids);
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForNodeIds(params Guid[] ids)
+        {
+            return _ignite.Cluster.ForNodeIds(ids);
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForPredicate(Func<IClusterNode, bool> p)
+        {
+            return _ignite.Cluster.ForPredicate(p);
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForAttribute(string name, string val)
+        {
+            return _ignite.Cluster.ForAttribute(name, val);
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForCacheNodes(string name)
+        {
+            return _ignite.Cluster.ForCacheNodes(name);
+        }
+        
+        /** <inheritdoc /> */
+        public IClusterGroup ForDataNodes(string name)
+        {
+            return _ignite.Cluster.ForDataNodes(name);
+        }
+        
+        /** <inheritdoc /> */
+        public IClusterGroup ForClientNodes(string name)
+        {
+            return _ignite.Cluster.ForClientNodes(name);
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForRemotes()
+        {
+            return _ignite.Cluster.ForRemotes();
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForHost(IClusterNode node)
+        {
+            return _ignite.Cluster.ForHost(node);
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForRandom()
+        {
+            return _ignite.Cluster.ForRandom();
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForOldest()
+        {
+            return _ignite.Cluster.ForOldest();
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForYoungest()
+        {
+            return _ignite.Cluster.ForYoungest();
+        }
+
+        /** <inheritdoc /> */
+        public IClusterGroup ForDotNet()
+        {
+            return _ignite.Cluster.ForDotNet();
+        }
+
+        /** <inheritdoc /> */
+        public ICollection<IClusterNode> Nodes()
+        {
+            return _ignite.Cluster.Nodes();
+        }
+
+        /** <inheritdoc /> */
+        public IClusterNode Node(Guid id)
+        {
+            return _ignite.Cluster.Node(id);
+        }
+
+        /** <inheritdoc /> */
+        public IClusterNode Node()
+        {
+            return _ignite.Cluster.Node();
+        }
+
+        /** <inheritdoc /> */
+        public IClusterMetrics Metrics()
+        {
+            return _ignite.Cluster.Metrics();
+        }
+
+        /** <inheritdoc /> */
+        public void Dispose()
+        {
+            _ignite.Dispose();
+        }
+
+        /** <inheritdoc /> */
+        public ICache<TK, TV> Cache<TK, TV>(string name)
+        {
+            return _ignite.Cache<TK, TV>(name);
+        }
+
+        /** <inheritdoc /> */
+        public ICache<TK, TV> GetOrCreateCache<TK, TV>(string name)
+        {
+            return _ignite.GetOrCreateCache<TK, TV>(name);
+        }
+
+        /** <inheritdoc /> */
+        public ICache<TK, TV> CreateCache<TK, TV>(string name)
+        {
+            return _ignite.CreateCache<TK, TV>(name);
+        }
+
+        /** <inheritdoc /> */
+        public IClusterNode LocalNode
+        {
+            get
+            {
+                return _ignite.Cluster.LocalNode;
+            }
+        }
+
+        /** <inheritdoc /> */
+        public bool PingNode(Guid nodeId)
+        {
+            return _ignite.Cluster.PingNode(nodeId);
+        }
+
+        /** <inheritdoc /> */
+        public long TopologyVersion
+        {
+            get { return _ignite.Cluster.TopologyVersion; }
+        }
+
+        /** <inheritdoc /> */
+        public ICollection<IClusterNode> Topology(long ver)
+        {
+            return _ignite.Cluster.Topology(ver);
+        }
+
+        /** <inheritdoc /> */
+        public void ResetMetrics()
+        {
+            _ignite.Cluster.ResetMetrics();
+        }
+
+        /** <inheritdoc /> */
+        public IDataStreamer<TK, TV> DataStreamer<TK, TV>(string cacheName)
+        {
+            return _ignite.DataStreamer<TK, TV>(cacheName);
+        }
+
+        /** <inheritdoc /> */
+        public IPortables Portables()
+        {
+            return _ignite.Portables();
+        }
+
+        /** <inheritdoc /> */
+        public ICacheAffinity Affinity(string name)
+        {
+            return _ignite.Affinity(name);
+        }
+
+        /** <inheritdoc /> */
+        public ITransactions Transactions
+        {
+            get { return _ignite.Transactions; }
+        }
+
+        /** <inheritdoc /> */
+        public IMessaging Message()
+        {
+            return _ignite.Message();
+        }
+
+        /** <inheritdoc /> */
+        public IMessaging Message(IClusterGroup clusterGroup)
+        {
+            return _ignite.Message(clusterGroup);
+        }
+
+        /** <inheritdoc /> */
+        public IEvents Events()
+        {
+            return _ignite.Events();
+        }
+
+        /** <inheritdoc /> */
+        public IEvents Events(IClusterGroup clusterGroup)
+        {
+            return _ignite.Events(clusterGroup);
+        }
+
+        /** <inheritdoc /> */
+        public IServices Services()
+        {
+            return _ignite.Services();
+        }
+
+        /** <inheritdoc /> */
+        public void WritePortable(IPortableWriter writer)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Target grid.
+        /// </summary>
+        internal IIgnite Target
+        {
+            get
+            {
+                return _ignite;
+            }
+        }
+
+        /** <inheritdoc /> */
+        public IPortableMetadata Metadata(int typeId)
+        {
+            return ((IClusterGroupEx)_ignite).Metadata(typeId);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs
new file mode 100644
index 0000000..265fd0d
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs
@@ -0,0 +1,438 @@
+/*
+ * 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
+{
+    using System;
+    using System.Collections.Generic;
+    using System.IO;
+    using System.Linq;
+    using System.Reflection;
+    using System.Runtime.InteropServices;
+    using System.Text;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Cluster;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Native utility methods.
+    /// </summary>
+    internal static class IgniteUtils
+    {
+        /** Environment variable: JAVA_HOME. */
+        private const string EnvJavaHome = "JAVA_HOME";
+
+        /** Directory: jre. */
+        private const string DirJre = "jre";
+
+        /** Directory: bin. */
+        private const string DirBin = "bin";
+
+        /** Directory: server. */
+        private const string DirServer = "server";
+
+        /** File: jvm.dll. */
+        private const string FileJvmDll = "jvm.dll";
+
+        /** File: Ignite.Common.dll. */
+        internal const string FileIgniteJniDll = "ignite.common.dll";
+        
+        /** Prefix for temp directory names. */
+        private const string DirIgniteTmp = "Ignite_";
+        
+        /** Loaded. */
+        private static bool _loaded;        
+
+        /** Thread-local random. */
+        [ThreadStatic]
+        private static Random _rnd;
+
+        /// <summary>
+        /// Initializes the <see cref="IgniteUtils"/> class.
+        /// </summary>
+        static IgniteUtils()
+        {
+            TryCleanTempDirectories();
+        }
+
+        /// <summary>
+        /// Gets thread local random.
+        /// </summary>
+        /// <returns>Thread local random.</returns>
+        public static Random ThreadLocalRandom()
+        {
+            if (_rnd == null)
+                _rnd = new Random();
+
+            return _rnd;
+        }
+
+        /// <summary>
+        /// Returns shuffled list copy.
+        /// </summary>
+        /// <returns>Shuffled list copy.</returns>
+        public static IList<T> Shuffle<T>(IList<T> list)
+        {
+            int cnt = list.Count;
+
+            if (cnt > 1) {
+                List<T> res = new List<T>(list);
+
+                Random rnd = ThreadLocalRandom();
+
+                while (cnt > 1)
+                {
+                    cnt--;
+                    
+                    int idx = rnd.Next(cnt + 1);
+
+                    T val = res[idx];
+                    res[idx] = res[cnt];
+                    res[cnt] = val;
+                }
+
+                return res;
+            }
+            return list;
+        }
+
+        /// <summary>
+        /// Load JVM DLL if needed.
+        /// </summary>
+        /// <param name="configJvmDllPath">JVM DLL path from config.</param>
+        public static void LoadDlls(string configJvmDllPath)
+        {
+            if (_loaded) return;
+
+            // 1. Load JNI dll.
+            LoadJvmDll(configJvmDllPath);
+
+            // 2. Load GG JNI dll.
+            UnmanagedUtils.Initialize();
+
+            _loaded = true;
+        }
+
+        /// <summary>
+        /// Create new instance of specified class.
+        /// </summary>
+        /// <param name="assemblyName">Assembly name.</param>
+        /// <param name="clsName">Class name</param>
+        /// <returns>New Instance.</returns>
+        public static object CreateInstance(string assemblyName, string clsName)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(clsName, "clsName");
+
+            var type = new TypeResolver().ResolveType(clsName, assemblyName);
+
+            if (type == null)
+                throw new IgniteException("Failed to create class instance [assemblyName=" + assemblyName +
+                    ", className=" + clsName + ']');
+
+            return Activator.CreateInstance(type);
+        }
+
+        /// <summary>
+        /// Set properties on the object.
+        /// </summary>
+        /// <param name="target">Target object.</param>
+        /// <param name="props">Properties.</param>
+        public static void SetProperties(object target, IEnumerable<KeyValuePair<string, object>> props)
+        {
+            if (props == null)
+                return;
+
+            IgniteArgumentCheck.NotNull(target, "target");
+
+            Type typ = target.GetType();
+
+            foreach (KeyValuePair<string, object> prop in props)
+            {
+                PropertyInfo prop0 = typ.GetProperty(prop.Key, 
+                    BindingFlags.Instance | BindingFlags.Public | BindingFlags.NonPublic);
+
+                if (prop0 == null)
+                    throw new IgniteException("Property is not found [type=" + typ.Name + 
+                        ", property=" + prop.Key + ']');
+
+                prop0.SetValue(target, prop.Value, null);
+            }
+        }
+
+        /// <summary>
+        /// Loads the JVM DLL.
+        /// </summary>
+        private static void LoadJvmDll(string configJvmDllPath)
+        {
+            var messages = new List<string>();
+            foreach (var dllPath in GetJvmDllPaths(configJvmDllPath))
+            {
+                var errCode = LoadDll(dllPath.Value, FileJvmDll);
+                if (errCode == 0)
+                    return;
+
+                messages.Add(string.Format("[option={0}, path={1}, errorCode={2}]", 
+                    dllPath.Key, dllPath.Value, errCode));
+
+                if (dllPath.Value == configJvmDllPath)
+                    break;  // if configJvmDllPath is specified and is invalid - do not try other options
+            }
+
+            if (!messages.Any())  // not loaded and no messages - everything was null
+                messages.Add(string.Format("Please specify IgniteConfiguration.JvmDllPath or {0}.", EnvJavaHome));
+
+            if (messages.Count == 1)
+                throw new IgniteException(string.Format("Failed to load {0} ({1})", FileJvmDll, messages[0]));
+
+            var combinedMessage = messages.Aggregate((x, y) => string.Format("{0}\n{1}", x, y));
+            throw new IgniteException(string.Format("Failed to load {0}:\n{1}", FileJvmDll, combinedMessage));
+        }
+
+        /// <summary>
+        /// Try loading DLLs first using file path, then using it's simple name.
+        /// </summary>
+        /// <param name="filePath"></param>
+        /// <param name="simpleName"></param>
+        /// <returns>Zero in case of success, error code in case of failure.</returns>
+        private static int LoadDll(string filePath, string simpleName)
+        {
+            int res = 0;
+
+            IntPtr ptr;
+
+            if (filePath != null)
+            {
+                ptr = NativeMethods.LoadLibrary(filePath);
+
+                if (ptr == IntPtr.Zero)
+                    res = Marshal.GetLastWin32Error();
+                else
+                    return res;
+            }
+
+            // Failed to load using file path, fallback to simple name.
+            ptr = NativeMethods.LoadLibrary(simpleName);
+
+            if (ptr == IntPtr.Zero)
+            {
+                // Preserve the first error code, if any.
+                if (res == 0)
+                    res = Marshal.GetLastWin32Error();
+            }
+            else
+                res = 0;
+
+            return res;
+        }
+
+        /// <summary>
+        /// Gets the JVM DLL paths in order of lookup priority.
+        /// </summary>
+        private static IEnumerable<KeyValuePair<string, string>> GetJvmDllPaths(string configJvmDllPath)
+        {
+            if (!string.IsNullOrEmpty(configJvmDllPath))
+                yield return new KeyValuePair<string, string>("IgniteConfiguration.JvmDllPath", configJvmDllPath);
+
+            var javaHomeDir = Environment.GetEnvironmentVariable(EnvJavaHome);
+
+            if (!string.IsNullOrEmpty(javaHomeDir))
+                yield return
+                    new KeyValuePair<string, string>(EnvJavaHome, GetJvmDllPath(Path.Combine(javaHomeDir, DirJre)));
+        }
+
+        /// <summary>
+        /// Gets the JVM DLL path from JRE dir.
+        /// </summary>
+        private static string GetJvmDllPath(string jreDir)
+        {
+            return Path.Combine(jreDir, DirBin, DirServer, FileJvmDll);
+        }
+
+        /// <summary>
+        /// Unpacks an embedded resource into a temporary folder and returns the full path of resulting file.
+        /// </summary>
+        /// <param name="resourceName">Resource name.</param>
+        /// <returns>Path to a temp file with an unpacked resource.</returns>
+        public static string UnpackEmbeddedResource(string resourceName)
+        {
+            var dllRes = Assembly.GetExecutingAssembly().GetManifestResourceNames()
+                .Single(x => x.EndsWith(resourceName, StringComparison.OrdinalIgnoreCase));
+
+            return WriteResourceToTempFile(dllRes, resourceName);
+        }
+
+        /// <summary>
+        /// Writes the resource to temporary file.
+        /// </summary>
+        /// <param name="resource">The resource.</param>
+        /// <param name="name">File name prefix</param>
+        /// <returns>Path to the resulting temp file.</returns>
+        private static string WriteResourceToTempFile(string resource, string name)
+        {
+            // Dll file name should not be changed, so we create a temp folder with random name instead.
+            var file = Path.Combine(GetTempDirectoryName(), name);
+
+            using (var src = Assembly.GetExecutingAssembly().GetManifestResourceStream(resource))
+            using (var dest = File.OpenWrite(file))
+            {
+                // ReSharper disable once PossibleNullReferenceException
+                src.CopyTo(dest);
+
+                return file;
+            }
+        }
+
+        /// <summary>
+        /// Tries to clean temporary directories created with <see cref="GetTempDirectoryName"/>.
+        /// </summary>
+        private static void TryCleanTempDirectories()
+        {
+            foreach (var dir in Directory.GetDirectories(Path.GetTempPath(), DirIgniteTmp + "*"))
+            {
+                try
+                {
+                    Directory.Delete(dir, true);
+                }
+                catch (IOException)
+                {
+                    // Expected
+                }
+                catch (UnauthorizedAccessException)
+                {
+                    // Expected
+                }
+            }
+        }
+
+        /// <summary>
+        /// Creates a uniquely named, empty temporary directory on disk and returns the full path of that directory.
+        /// </summary>
+        /// <returns>The full path of the temporary directory.</returns>
+        private static string GetTempDirectoryName()
+        {
+            while (true)
+            {
+                var dir = Path.Combine(Path.GetTempPath(), DirIgniteTmp + Path.GetRandomFileName());
+
+                try
+                {
+                    return Directory.CreateDirectory(dir).FullName;
+                }
+                catch (IOException)
+                {
+                    // Expected
+                }
+                catch (UnauthorizedAccessException)
+                {
+                    // Expected
+                }
+            }
+        }
+
+        /// <summary>
+        /// Convert unmanaged char array to string.
+        /// </summary>
+        /// <param name="chars">Char array.</param>
+        /// <param name="charsLen">Char array length.</param>
+        /// <returns></returns>
+        public static unsafe string Utf8UnmanagedToString(sbyte* chars, int charsLen)
+        {
+            IntPtr ptr = new IntPtr(chars);
+
+            if (ptr == IntPtr.Zero)
+                return null;
+
+            byte[] arr = new byte[charsLen];
+
+            Marshal.Copy(ptr, arr, 0, arr.Length);
+
+            return Encoding.UTF8.GetString(arr);
+        }
+
+        /// <summary>
+        /// Convert string to unmanaged byte array.
+        /// </summary>
+        /// <param name="str">String.</param>
+        /// <returns>Unmanaged byte array.</returns>
+        public static unsafe sbyte* StringToUtf8Unmanaged(string str)
+        {
+            var ptr = IntPtr.Zero;
+
+            if (str != null)
+            {
+                byte[] strBytes = Encoding.UTF8.GetBytes(str);
+
+                ptr = Marshal.AllocHGlobal(strBytes.Length + 1);
+
+                Marshal.Copy(strBytes, 0, ptr, strBytes.Length);
+
+                *((byte*)ptr.ToPointer() + strBytes.Length) = 0; // NULL-terminator.
+            }
+            
+            return (sbyte*)ptr.ToPointer();
+        }
+
+        /// <summary>
+        /// Reads node collection from stream.
+        /// </summary>
+        /// <param name="reader">Reader.</param>
+        /// <param name="pred">The predicate.</param>
+        /// <returns> Nodes list or null. </returns>
+        public static List<IClusterNode> ReadNodes(IPortableRawReader reader, Func<ClusterNodeImpl, bool> pred = null)
+        {
+            var cnt = reader.ReadInt();
+
+            if (cnt < 0)
+                return null;
+
+            var res = new List<IClusterNode>(cnt);
+
+            var ignite = ((PortableReaderImpl)reader).Marshaller.Ignite;
+
+            if (pred == null)
+            {
+                for (var i = 0; i < cnt; i++)
+                    res.Add(ignite.GetNode(reader.ReadGuid()));
+            }
+            else
+            {
+                for (var i = 0; i < cnt; i++)
+                {
+                    var node = ignite.GetNode(reader.ReadGuid());
+                    
+                    if (pred(node))
+                        res.Add(node);
+                }
+            }
+
+            return res;
+        }
+
+        /// <summary>
+        /// Gets the asynchronous mode disabled exception.
+        /// </summary>
+        /// <returns>Asynchronous mode disabled exception.</returns>
+        public static InvalidOperationException GetAsyncModeDisabledException()
+        {
+            return new InvalidOperationException("Asynchronous mode is disabled");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Interop/InteropDotNetConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Interop/InteropDotNetConfiguration.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Interop/InteropDotNetConfiguration.cs
new file mode 100644
index 0000000..2dffd28
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Interop/InteropDotNetConfiguration.cs
@@ -0,0 +1,62 @@
+/*
+ * 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.Interop
+{
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// .Net configuration as defined in Java configuration file.
+    /// </summary>
+    internal class InteropDotNetConfiguration : IPortableWriteAware
+    {
+        /// <summary>
+        /// Portable configuration.
+        /// </summary>
+        public InteropDotNetPortableConfiguration PortableCfg { get; set; }
+
+        /// <summary>
+        /// Assemblies to load.
+        /// </summary>
+        public IList<string> Assemblies { get; set; }
+
+        /** {@inheritDoc} */
+        public void WritePortable(IPortableWriter writer)
+        {
+            IPortableRawWriter rawWriter = writer.RawWriter();
+
+            rawWriter.WriteObject(PortableCfg);
+
+            rawWriter.WriteGenericCollection(Assemblies);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="InteropDotNetConfiguration"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        public InteropDotNetConfiguration(IPortableReader reader)
+        {
+            IPortableRawReader rawReader = reader.RawReader();
+
+            PortableCfg = rawReader.ReadObject<InteropDotNetPortableConfiguration>();
+
+            Assemblies = (List<string>) rawReader.ReadGenericCollection<string>();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Interop/InteropDotNetPortableConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Interop/InteropDotNetPortableConfiguration.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Interop/InteropDotNetPortableConfiguration.cs
new file mode 100644
index 0000000..4f36e84
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Interop/InteropDotNetPortableConfiguration.cs
@@ -0,0 +1,127 @@
+/*
+ * 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.Interop
+{
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// .Net portable configuration as defined in Java configuration.
+    /// </summary>
+    internal class InteropDotNetPortableConfiguration : IPortableWriteAware
+    {
+        /// <summary>
+        /// Type configurations.
+        /// </summary>
+        public ICollection<InteropDotNetPortableTypeConfiguration> TypeConfigurations { get; set; }
+
+        /// <summary>
+        /// Portable types. Shorthand for creating PortableTypeConfiguration.
+        /// </summary>
+        public ICollection<string> Types { get; set; }
+
+        /// <summary>
+        /// Default name mapper.
+        /// </summary>
+        public string DefaultNameMapper { get; set; }
+
+        /// <summary>
+        /// Default ID mapper.
+        /// </summary>
+        public string DefaultIdMapper { get; set; }
+
+        /// <summary>
+        /// Default serializer.
+        /// </summary>
+        public string DefaultSerializer { get; set; }
+
+        /// <summary>
+        /// Default metadata enabled flag. Defaults to true.
+        /// </summary>
+        public bool DefaultMetadataEnabled { get; set; }
+
+        /// <summary>
+        /// Keep deserialized flag. If set to non-null value, overrides default value set in 
+        /// PortableConfiguration.
+        /// </summary>
+        public bool DefaultKeepDeserialized { get; set; }
+
+        /// <summary>
+        /// Creates PortableConfiguration.
+        /// </summary>
+        /// <returns>PortableConfiguration</returns>
+        public PortableConfiguration ToPortableConfiguration()
+        {
+            PortableConfiguration res = new PortableConfiguration();
+
+            if (TypeConfigurations != null)
+            {
+                List<PortableTypeConfiguration> typeCfgs = new List<PortableTypeConfiguration>();
+
+                foreach (InteropDotNetPortableTypeConfiguration dotNetTypeCfg in TypeConfigurations)
+                    typeCfgs.Add(dotNetTypeCfg.ToPortableTypeConfiguration());
+
+                res.TypeConfigurations = typeCfgs;
+            }
+
+            res.Types = Types;
+            res.DefaultNameMapper =
+                (IPortableNameMapper) InteropDotNetPortableTypeConfiguration.CreateInstance(DefaultNameMapper);
+            res.DefaultIdMapper =
+                (IPortableIdMapper) InteropDotNetPortableTypeConfiguration.CreateInstance(DefaultIdMapper);
+            res.DefaultSerializer =
+                (IPortableSerializer) InteropDotNetPortableTypeConfiguration.CreateInstance(DefaultSerializer);
+            res.DefaultMetadataEnabled = DefaultMetadataEnabled;
+            res.DefaultKeepDeserialized = DefaultKeepDeserialized;
+
+            return res;
+        }
+
+        /** {@inheritDoc} */
+        public void WritePortable(IPortableWriter writer)
+        {
+            IPortableRawWriter rawWriter = writer.RawWriter();
+
+            rawWriter.WriteGenericCollection(TypeConfigurations);
+            rawWriter.WriteGenericCollection(Types);
+            rawWriter.WriteString(DefaultNameMapper);
+            rawWriter.WriteString(DefaultIdMapper);
+            rawWriter.WriteString(DefaultSerializer);
+            rawWriter.WriteBoolean(DefaultMetadataEnabled);
+            rawWriter.WriteBoolean(DefaultKeepDeserialized);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="InteropDotNetPortableConfiguration"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        public InteropDotNetPortableConfiguration(IPortableReader reader)
+        {
+            IPortableRawReader rawReader = reader.RawReader();
+
+            TypeConfigurations = rawReader.ReadGenericCollection<InteropDotNetPortableTypeConfiguration>();
+            Types = rawReader.ReadGenericCollection<string>();
+            DefaultNameMapper = rawReader.ReadString();
+            DefaultIdMapper = rawReader.ReadString();
+            DefaultSerializer = rawReader.ReadString();
+            DefaultMetadataEnabled = rawReader.ReadBoolean();
+            DefaultKeepDeserialized = rawReader.ReadBoolean();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Interop/InteropDotNetPortableTypeConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Interop/InteropDotNetPortableTypeConfiguration.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Interop/InteropDotNetPortableTypeConfiguration.cs
new file mode 100644
index 0000000..0c3b433
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Interop/InteropDotNetPortableTypeConfiguration.cs
@@ -0,0 +1,151 @@
+/*
+ * 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.Interop
+{
+    using System;
+    using System.Reflection;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// .Net portable configuration type as defined in Java configuration.
+    /// </summary>
+    internal class InteropDotNetPortableTypeConfiguration : IPortableWriteAware
+    {
+        /// <summary>
+        /// Assembly name.
+        /// </summary>
+        public string AssemblyName { get; set; }
+
+        /// <summary>
+        /// Fully qualified type name.
+        /// </summary>
+        public string TypeName { get; set; }
+
+        /// <summary>
+        /// Name mapper for the given type.
+        /// </summary>
+        public string NameMapper { get; set; }
+
+        /// <summary>
+        /// ID mapper for the given type. When it is necessary to resolve class (field) ID, then
+        /// this property will be checked first. If not set, then PortableClassIdAttribute
+        /// (PortableFieldIdAttribute) will be checked in class through reflection. If required
+        /// attribute is not set, then ID will be hash code of the class (field) simple name in lower case.
+        /// </summary>
+        public string IdMapper { get; set; }
+
+        /// <summary>
+        /// Serializer for the given type. If not provided and class implements IPortable
+        /// then its custom logic will be used. If not provided and class doesn't implement IPortable
+        /// then all fields of the class except of those with [NotSerialized] attribute will be serialized
+        ///with help of reflection.
+        /// </summary>
+        public string Serializer { get; set; }
+
+        /// <summary>
+        /// Affinity key field name.
+        /// </summary>
+        public string AffinityKeyFieldName { get; set; }
+
+        /// <summary>
+        /// Metadata enabled flag. If set to non-null value, overrides default value set in
+        /// PortableConfiguration.
+        /// </summary>
+        public bool? MetadataEnabled { get; set; }
+
+        /// <summary>
+        /// Keep deserialized flag. If set to non-null value, overrides default value set in 
+        /// PortableConfiguration.
+        /// </summary>
+        public bool? KeepDeserialized { get; set; }
+
+        /// <summary>
+        /// Creates new instance of PortableTypeConfiguration.
+        /// </summary>
+        /// <returns>PortableTypeConfiguration</returns>
+        public PortableTypeConfiguration ToPortableTypeConfiguration()
+        {
+            return new PortableTypeConfiguration
+            {
+                AssemblyName = AssemblyName,
+                AffinityKeyFieldName = AffinityKeyFieldName,
+                TypeName = TypeName,
+                NameMapper = (IPortableNameMapper) CreateInstance(NameMapper),
+                IdMapper = (IPortableIdMapper) CreateInstance(IdMapper),
+                Serializer = (IPortableSerializer) CreateInstance(Serializer),
+                MetadataEnabled = MetadataEnabled,
+                KeepDeserialized = KeepDeserialized
+            };
+        }
+
+        /** {@inheritDoc} */
+        public void WritePortable(IPortableWriter writer)
+        {
+            IPortableRawWriter rawWriter = writer.RawWriter();
+
+            rawWriter.WriteString(AssemblyName);
+            rawWriter.WriteString(TypeName);
+            rawWriter.WriteString(NameMapper);
+            rawWriter.WriteString(IdMapper);
+            rawWriter.WriteString(Serializer);
+            rawWriter.WriteString(AffinityKeyFieldName);
+            rawWriter.WriteObject(MetadataEnabled);
+            rawWriter.WriteObject(KeepDeserialized);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="InteropDotNetPortableTypeConfiguration"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        public InteropDotNetPortableTypeConfiguration(IPortableReader reader)
+        {
+            IPortableRawReader rawReader = reader.RawReader();
+
+            AssemblyName = rawReader.ReadString();
+            TypeName = rawReader.ReadString();
+            NameMapper = rawReader.ReadString();
+            IdMapper = rawReader.ReadString();
+            Serializer = rawReader.ReadString();
+            AffinityKeyFieldName = rawReader.ReadString();
+            MetadataEnabled = rawReader.ReadObject<bool?>();
+            KeepDeserialized = rawReader.ReadObject<bool?>();
+        }
+
+        /// <summary>
+        /// Create new instance of specified class.
+        /// </summary>
+        /// <param name="typeName">Name of the type.</param>
+        /// <returns>New Instance.</returns>
+        public static object CreateInstance(string typeName)
+        {
+            if (typeName == null)
+                return null;
+
+            foreach (Assembly assembly in AppDomain.CurrentDomain.GetAssemblies())
+            {
+                object instance = assembly.CreateInstance(typeName);
+
+                if (instance != null)
+                    return instance;
+            }
+
+            throw new PortableException("Failed to find class: " + typeName);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/InteropExceptionHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/InteropExceptionHolder.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/InteropExceptionHolder.cs
new file mode 100644
index 0000000..98d57da
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/InteropExceptionHolder.cs
@@ -0,0 +1,85 @@
+/*
+ * 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
+{
+    using System;
+    using System.Runtime.Serialization.Formatters.Binary;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Holder of exception which must be serialized to Java and then backwards to the native platform.
+    /// </summary>
+    internal class InteropExceptionHolder : IPortableMarshalAware
+    {
+        /** Initial exception. */
+        private Exception _err;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        public InteropExceptionHolder()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="err">Error.</param>
+        public InteropExceptionHolder(Exception err)
+        {
+            _err = err;
+        }
+
+        /// <summary>
+        /// Underlying exception.
+        /// </summary>
+        public Exception Error
+        {
+            get { return _err; }
+        }
+
+        /** <inheritDoc /> */
+        public void WritePortable(IPortableWriter writer)
+        {
+            var writer0 = (PortableWriterImpl) writer.RawWriter();
+
+            if (writer0.IsPortable(_err))
+            {
+                writer0.WriteBoolean(true);
+                writer0.WriteObject(_err);
+            }
+            else
+            {
+                writer0.WriteBoolean(false);
+
+                BinaryFormatter bf = new BinaryFormatter();
+
+                bf.Serialize(new PortableStreamAdapter(writer0.Stream), _err);
+            }
+        }
+
+        /** <inheritDoc /> */
+        public void ReadPortable(IPortableReader reader)
+        {
+            throw new NotImplementedException();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/LifecycleBeanHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/LifecycleBeanHolder.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/LifecycleBeanHolder.cs
new file mode 100644
index 0000000..cce4ec5
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/LifecycleBeanHolder.cs
@@ -0,0 +1,66 @@
+/*
+ * 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
+{
+    using Apache.Ignite.Core.Impl.Resource;
+    using Apache.Ignite.Core.Lifecycle;
+
+    /// <summary>
+    /// Lifecycle bean holder.
+    /// </summary>
+    internal class LifecycleBeanHolder : ILifecycleBean
+    {
+        /** Target bean. */
+        private readonly ILifecycleBean _target;
+
+        /** Whether start event was invoked. */
+        private volatile bool _startEvt;
+        
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="target">Target bean.</param>
+        public LifecycleBeanHolder(ILifecycleBean target)
+        {
+            _target = target;
+        }
+
+        /** <inheritDoc /> */
+        public void OnLifecycleEvent(LifecycleEventType evt)
+        {
+            if (evt == LifecycleEventType.AfterNodeStart)
+                // This event cannot be propagated right away because at this point we
+                // do not have Ignite instance yet. So just schedule it.
+                _startEvt = true;
+            else
+                _target.OnLifecycleEvent(evt);
+        }
+
+        /// <summary>
+        /// Grid start callback.
+        /// </summary>
+        /// <param name="grid">Ignite instance.</param>
+        internal void OnStart(Ignite grid)
+        {
+            ResourceProcessor.Inject(_target, grid);
+
+            if (_startEvt)
+                _target.OnLifecycleEvent(LifecycleEventType.AfterNodeStart);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/InteropExternalMemory.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/InteropExternalMemory.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/InteropExternalMemory.cs
new file mode 100644
index 0000000..d356b5e
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/InteropExternalMemory.cs
@@ -0,0 +1,46 @@
+/*
+ * 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.Memory
+{
+    /// <summary>
+    /// Interop external memory chunk.
+    /// </summary>
+    internal class InteropExternalMemory : PlatformMemory
+    {
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="memPtr">Memory pointer.</param>
+        public InteropExternalMemory(long memPtr) : base(memPtr)
+        {
+            // No-op.
+        }
+
+        /** <inheritdoc /> */
+        public override void Reallocate(int cap)
+        {
+            InteropMemoryUtils.ReallocateExternal(Pointer, cap);
+        }
+
+        /** <inheritdoc /> */
+        public override void Release()
+        {
+            // Memory can only be released by native platform.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/InteropMemoryUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/InteropMemoryUtils.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/InteropMemoryUtils.cs
new file mode 100644
index 0000000..485d3db
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/InteropMemoryUtils.cs
@@ -0,0 +1,38 @@
+/*
+ * 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.Memory
+{
+    using Apache.Ignite.Core.Impl.Unmanaged;
+
+    /// <summary>
+    /// Utility methods for interop memory management.
+    /// </summary>
+    internal static class InteropMemoryUtils
+    {
+        /// <summary>
+        /// Re-allocate external memory chunk.
+        /// </summary>
+        /// <param name="memPtr">Memory pointer.</param>
+        /// <param name="cap">CalculateCapacity.</param>
+        /// <returns>New memory pointer.</returns>
+        public static void ReallocateExternal(long memPtr, int cap)
+        {
+            UnmanagedUtils.Reallocate(memPtr, cap);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryManager.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryManager.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryManager.cs
index 2d52dd6..3dc3953 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryManager.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryManager.cs
@@ -17,10 +17,9 @@
 
 namespace Apache.Ignite.Core.Impl.Memory
 {
-    using System;
     using System.Diagnostics.CodeAnalysis;
     using System.Threading;
-    
+
     /// <summary>
     /// Memory manager implementation.
     /// </summary>
@@ -100,7 +99,7 @@ namespace Apache.Ignite.Core.Impl.Memory
         /// <returns>Memory.</returns>
         protected virtual IPlatformMemory GetExternalMemory(long memPtr)
         {
-            throw new NotSupportedException("Not supported in Ignite yet");
+            return new InteropExternalMemory(memPtr);
         }
     }
 }


[38/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
IGNITE-1348: Moved GridGain's .Net module to Ignite.


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

Branch: refs/heads/ignite-843
Commit: 5cec202cbe029ca563c112c4916b25605930ceea
Parents: 4d3c1f0
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Fri Sep 4 16:31:55 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Sep 4 16:31:59 2015 +0300

----------------------------------------------------------------------
 .../internal/portable/PortableContext.java      |    7 +
 .../dotnet/PlatformDotNetConfiguration.java     |  119 +
 .../PlatformDotNetPortableConfiguration.java    |  228 ++
 ...PlatformDotNetPortableTypeConfiguration.java |  248 ++
 modules/platform/pom.xml                        |   14 +
 .../Apache.Ignite.Core.csproj                   |  283 +-
 .../Cache/CacheAtomicUpdateTimeoutException.cs  |   67 +
 .../Cache/CacheEntryProcessorException.cs       |   79 +
 .../Apache.Ignite.Core/Cache/CacheException.cs  |   68 +
 .../Cache/CachePartialUpdateException.cs        |  119 +
 .../Apache.Ignite.Core/Cache/CachePeekMode.cs   |   68 +
 .../Cache/Event/CacheEntryEventType.cs          |   41 +
 .../Cache/Event/ICacheEntryEvent.cs             |   40 +
 .../Cache/Event/ICacheEntryEventFilter.cs       |   31 +
 .../Cache/Event/ICacheEntryEventListener.cs     |   33 +
 .../Cache/Expiry/ExpiryPolicy.cs                |   89 +
 .../Cache/Expiry/IExpiryPolicy.cs               |   59 +
 .../dotnet/Apache.Ignite.Core/Cache/ICache.cs   |  542 +++
 .../Apache.Ignite.Core/Cache/ICacheAffinity.cs  |  161 +
 .../Apache.Ignite.Core/Cache/ICacheEntry.cs     |   37 +
 .../Cache/ICacheEntryFilter.cs                  |   34 +
 .../Cache/ICacheEntryProcessor.cs               |   45 +
 .../Cache/ICacheEntryProcessorResult.cs         |   40 +
 .../Apache.Ignite.Core/Cache/ICacheLock.cs      |   58 +
 .../Apache.Ignite.Core/Cache/ICacheMetrics.cs   |  486 +++
 .../Cache/IMutableCacheEntry.cs                 |   47 +
 .../Cache/Query/Continuous/ContinuousQuery.cs   |  170 +
 .../Query/Continuous/IContinuousQueryHandle.cs  |   51 +
 .../Cache/Query/IQueryCursor.cs                 |   40 +
 .../Apache.Ignite.Core/Cache/Query/QueryBase.cs |   82 +
 .../Apache.Ignite.Core/Cache/Query/ScanQuery.cs |   77 +
 .../Cache/Query/SqlFieldsQuery.cs               |   81 +
 .../Apache.Ignite.Core/Cache/Query/SqlQuery.cs  |  119 +
 .../Apache.Ignite.Core/Cache/Query/TextQuery.cs |  104 +
 .../Store/CacheParallelLoadStoreAdapter.cs      |  205 ++
 .../Cache/Store/CacheStoreAdapter.cs            |  146 +
 .../Cache/Store/CacheStoreException.cs          |   66 +
 .../Cache/Store/ICacheStore.cs                  |  184 +
 .../Cache/Store/ICacheStoreSession.cs           |   42 +
 .../Cluster/ClusterGroupEmptyException.cs       |   70 +
 .../Cluster/ClusterTopologyException.cs         |   69 +
 .../Apache.Ignite.Core/Cluster/ICluster.cs      |   80 +
 .../Apache.Ignite.Core/Cluster/IClusterGroup.cs |  229 ++
 .../Cluster/IClusterMetrics.cs                  |  515 +++
 .../Apache.Ignite.Core/Cluster/IClusterNode.cs  |  138 +
 .../Cluster/IClusterNodeFilter.cs               |   32 +
 .../Apache.Ignite.Core/Common/IAsyncSupport.cs  |    2 +-
 .../Common/IgniteException.cs                   |    2 +-
 .../Apache.Ignite.Core/Common/IgniteGuid.cs     |  138 +
 .../ComputeExecutionRejectedException.cs        |   69 +
 .../Compute/ComputeJobAdapter.cs                |  122 +
 .../Compute/ComputeJobFailoverException.cs      |   72 +
 .../Compute/ComputeJobResultPolicy.cs           |   45 +
 .../Compute/ComputeTaskAdapter.cs               |   93 +
 .../Compute/ComputeTaskCancelledException.cs    |   69 +
 .../ComputeTaskNoResultCacheAttribute.cs        |   35 +
 .../Compute/ComputeTaskSplitAdapter.cs          |   95 +
 .../Compute/ComputeTaskTimeoutException.cs      |   67 +
 .../Compute/ComputeUserUndeclaredException.cs   |   70 +
 .../Apache.Ignite.Core/Compute/ICompute.cs      |  274 ++
 .../Apache.Ignite.Core/Compute/IComputeFunc.cs  |   55 +
 .../Apache.Ignite.Core/Compute/IComputeJob.cs   |   58 +
 .../Compute/IComputeJobResult.cs                |   73 +
 .../Compute/IComputeReducer.cs                  |   39 +
 .../Apache.Ignite.Core/Compute/IComputeTask.cs  |  132 +
 .../Datastream/IDataStreamer.cs                 |  206 ++
 .../Datastream/IStreamReceiver.cs               |   38 +
 .../Datastream/StreamTransformer.cs             |   73 +
 .../Datastream/StreamVisitor.cs                 |   55 +
 .../Apache.Ignite.Core/Events/CacheEvent.cs     |  176 +
 .../Events/CacheQueryExecutedEvent.cs           |   97 +
 .../Events/CacheQueryReadEvent.cs               |  134 +
 .../Events/CacheRebalancingEvent.cs             |   98 +
 .../Events/CheckpointEvent.cs                   |   50 +
 .../Apache.Ignite.Core/Events/DiscoveryEvent.cs |   80 +
 .../Apache.Ignite.Core/Events/EventBase.cs      |  160 +
 .../Apache.Ignite.Core/Events/EventReader.cs    |   72 +
 .../Apache.Ignite.Core/Events/EventType.cs      |  514 +++
 .../dotnet/Apache.Ignite.Core/Events/IEvent.cs  |   74 +
 .../Apache.Ignite.Core/Events/IEventFilter.cs   |   36 +
 .../dotnet/Apache.Ignite.Core/Events/IEvents.cs |  182 +
 .../Apache.Ignite.Core/Events/JobEvent.cs       |  100 +
 .../Apache.Ignite.Core/Events/SwapSpaceEvent.cs |   50 +
 .../Apache.Ignite.Core/Events/TaskEvent.cs      |   91 +
 .../main/dotnet/Apache.Ignite.Core/IIgnite.cs   |  168 +
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  140 +
 .../main/dotnet/Apache.Ignite.Core/Ignition.cs  |  626 +++-
 .../Impl/Cache/CacheAffinityImpl.cs             |  275 ++
 .../Apache.Ignite.Core/Impl/Cache/CacheEntry.cs |  126 +
 .../Impl/Cache/CacheEntryFilterHolder.cs        |  147 +
 .../Impl/Cache/CacheEntryProcessorHolder.cs     |  145 +
 .../Impl/Cache/CacheEntryProcessorResult.cs     |   65 +
 .../Cache/CacheEntryProcessorResultHolder.cs    |  127 +
 .../Impl/Cache/CacheEnumerable.cs               |   82 +
 .../Impl/Cache/CacheEnumerator.cs               |  117 +
 .../Impl/Cache/CacheEnumeratorProxy.cs          |  156 +
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |  932 +++++
 .../Apache.Ignite.Core/Impl/Cache/CacheLock.cs  |  171 +
 .../Impl/Cache/CacheMetricsImpl.cs              |  248 ++
 .../Apache.Ignite.Core/Impl/Cache/CacheOp.cs    |   63 +
 .../Impl/Cache/CacheProxyImpl.cs                |  499 +++
 .../Impl/Cache/Event/CacheEntryCreateEvent.cs   |   74 +
 .../Impl/Cache/Event/CacheEntryRemoveEvent.cs   |   74 +
 .../Impl/Cache/Event/CacheEntryUpdateEvent.cs   |   79 +
 .../Impl/Cache/MutableCacheEntry.cs             |  163 +
 .../Impl/Cache/Query/AbstractQueryCursor.cs     |  264 ++
 .../Query/Continuous/ContinuousQueryFilter.cs   |  125 +
 .../Continuous/ContinuousQueryFilterHolder.cs   |  118 +
 .../Continuous/ContinuousQueryHandleImpl.cs     |  216 ++
 .../Query/Continuous/ContinuousQueryUtils.cs    |  115 +
 .../Impl/Cache/Query/FieldsQueryCursor.cs       |   54 +
 .../Impl/Cache/Query/QueryCursor.cs             |   50 +
 .../Impl/Cache/Store/CacheStore.cs              |  263 ++
 .../Impl/Cache/Store/CacheStoreSession.cs       |   53 +
 .../Impl/Cache/Store/CacheStoreSessionProxy.cs  |   63 +
 .../Impl/Cluster/ClusterGroupImpl.cs            |  577 +++
 .../Impl/Cluster/ClusterMetricsImpl.cs          |  292 ++
 .../Impl/Cluster/ClusterNodeImpl.cs             |  221 ++
 .../Impl/Cluster/IClusterGroupEx.cs             |   35 +
 .../Impl/Common/DelegateTypeDescriptor.cs       |  327 ++
 .../Apache.Ignite.Core/Impl/Common/Future.cs    |    3 +-
 .../Impl/Common/FutureConverter.cs              |   62 +
 .../Impl/Common/GridArgumentCheck.cs            |   76 -
 .../Impl/Common/IgniteArgumentCheck.cs          |   76 +
 .../Impl/Common/PortableResultWrapper.cs        |   68 +
 .../Closure/ComputeAbstractClosureTask.cs       |  101 +
 .../Impl/Compute/Closure/ComputeActionJob.cs    |   83 +
 .../Impl/Compute/Closure/ComputeFuncJob.cs      |   89 +
 .../Compute/Closure/ComputeMultiClosureTask.cs  |   56 +
 .../Impl/Compute/Closure/ComputeOutFuncJob.cs   |   76 +
 .../Closure/ComputeReducingClosureTask.cs       |   61 +
 .../Compute/Closure/ComputeSingleClosureTask.cs |   48 +
 .../Compute/Closure/IComputeResourceInjector.cs |   31 +
 .../Apache.Ignite.Core/Impl/Compute/Compute.cs  |  213 ++
 .../Impl/Compute/ComputeAsync.cs                |  261 ++
 .../Impl/Compute/ComputeFunc.cs                 |  119 +
 .../Impl/Compute/ComputeImpl.cs                 |  645 ++++
 .../Impl/Compute/ComputeJob.cs                  |  163 +
 .../Impl/Compute/ComputeJobHolder.cs            |  240 ++
 .../Compute/ComputeJobResultGenericWrapper.cs   |   70 +
 .../Impl/Compute/ComputeJobResultImpl.cs        |   96 +
 .../Impl/Compute/ComputeOutFunc.cs              |  123 +
 .../Impl/Compute/ComputeTaskHolder.cs           |  484 +++
 .../Impl/Datastream/DataStreamerBatch.cs        |  269 ++
 .../Impl/Datastream/DataStreamerEntry.cs        |   64 +
 .../Impl/Datastream/DataStreamerImpl.cs         |  832 +++++
 .../Impl/Datastream/DataStreamerRemoveEntry.cs  |   48 +
 .../Impl/Datastream/StreamReceiverHolder.cs     |  144 +
 .../Apache.Ignite.Core/Impl/Events/Events.cs    |  498 +++
 .../Impl/Events/EventsAsync.cs                  |  158 +
 .../Impl/Events/RemoteListenEventFilter.cs      |   85 +
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs   |  204 ++
 .../Impl/Handle/HandleRegistry.cs               |    4 +-
 .../Apache.Ignite.Core/Impl/IInteropCallback.cs |   34 +
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |  547 +++
 .../Impl/IgniteConfigurationEx.cs               |   57 +
 .../Apache.Ignite.Core/Impl/IgniteManager.cs    |  492 +++
 .../Apache.Ignite.Core/Impl/IgniteProxy.cs      |  351 ++
 .../Apache.Ignite.Core/Impl/IgniteUtils.cs      |  438 +++
 .../Impl/Interop/InteropDotNetConfiguration.cs  |   62 +
 .../InteropDotNetPortableConfiguration.cs       |  127 +
 .../InteropDotNetPortableTypeConfiguration.cs   |  151 +
 .../Impl/InteropExceptionHolder.cs              |   85 +
 .../Impl/LifecycleBeanHolder.cs                 |   66 +
 .../Impl/Memory/InteropExternalMemory.cs        |   46 +
 .../Impl/Memory/InteropMemoryUtils.cs           |   38 +
 .../Impl/Memory/PlatformMemoryManager.cs        |    5 +-
 .../Impl/Memory/PlatformMemoryStream.cs         |    3 +-
 .../Impl/Memory/PlatformMemoryUtils.cs          |    2 +-
 .../Impl/Memory/PlatformPooledMemory.cs         |    2 +-
 .../Impl/Messaging/MessageFilterHolder.cs       |  179 +
 .../Impl/Messaging/Messaging.cs                 |  262 ++
 .../Impl/Messaging/MessagingAsync.cs            |   68 +
 .../Apache.Ignite.Core/Impl/NativeMethods.cs    |   47 +
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |  715 ++++
 .../Portable/IPortableSystemTypeSerializer.cs   |   34 +
 .../Impl/Portable/IPortableTypeDescriptor.cs    |  124 +
 .../Impl/Portable/IPortableWriteAware.cs        |   34 +
 .../Impl/Portable/Io/PortableAbstractStream.cs  | 1299 +++++++
 .../Impl/Portable/Io/PortableHeapStream.cs      |  447 +++
 .../Impl/Portable/Io/PortableStreamAdapter.cs   |  114 +
 .../Metadata/IPortableMetadataHandler.cs        |   41 +
 .../Metadata/PortableHashsetMetadataHandler.cs  |   69 +
 .../Portable/Metadata/PortableMetadataHolder.cs |  149 +
 .../Portable/Metadata/PortableMetadataImpl.cs   |  200 ++
 .../Impl/Portable/PortableBuilderField.cs       |   73 +
 .../Impl/Portable/PortableBuilderImpl.cs        |  923 +++++
 .../Impl/Portable/PortableCollectionInfo.cs     |  251 ++
 .../Impl/Portable/PortableFullTypeDescriptor.cs |  203 ++
 .../Impl/Portable/PortableHandleDictionary.cs   |  187 +
 .../Portable/PortableMarshalAwareSerializer.cs  |   45 +
 .../Impl/Portable/PortableMarshaller.cs         |  603 ++++
 .../Impl/Portable/PortableMode.cs               |   40 +
 .../Impl/Portable/PortableObjectHandle.cs       |   59 +
 .../PortableOrSerializableObjectHolder.cs       |   66 +
 .../Portable/PortableReaderHandleDictionary.cs  |   42 +
 .../Impl/Portable/PortableReaderImpl.cs         | 1013 ++++++
 .../Impl/Portable/PortableReflectiveRoutines.cs |  483 +++
 .../Portable/PortableReflectiveSerializer.cs    |  218 ++
 .../Portable/PortableSurrogateTypeDescriptor.cs |  133 +
 .../Impl/Portable/PortableSystemHandlers.cs     | 1336 +++++++
 .../Portable/PortableSystemTypeSerializer.cs    |   62 +
 .../Impl/Portable/PortableUserObject.cs         |  385 ++
 .../Impl/Portable/PortableUtils.cs              | 2039 +++++++++++
 .../Impl/Portable/PortableWriterImpl.cs         | 1305 +++++++
 .../Impl/Portable/PortablesImpl.cs              |  205 ++
 .../Impl/Portable/SerializableObjectHolder.cs   |   66 +
 .../Impl/Portable/TypeResolver.cs               |  227 ++
 .../Impl/Resource/IResourceInjector.cs          |   27 +
 .../Impl/Resource/ResourceFieldInjector.cs      |   47 +
 .../Impl/Resource/ResourceMethodInjector.cs     |   48 +
 .../Impl/Resource/ResourceProcessor.cs          |  105 +
 .../Impl/Resource/ResourcePropertyInjector.cs   |   47 +
 .../Impl/Resource/ResourceTypeDescriptor.cs     |  291 ++
 .../Impl/Services/ServiceContext.cs             |   60 +
 .../Impl/Services/ServiceDescriptor.cs          |  106 +
 .../Impl/Services/ServiceProxy.cs               |   71 +
 .../Impl/Services/ServiceProxyInvoker.cs        |  136 +
 .../Impl/Services/ServiceProxySerializer.cs     |  140 +
 .../Impl/Services/Services.cs                   |  316 ++
 .../Impl/Services/ServicesAsync.cs              |   89 +
 .../Impl/Transactions/AsyncTransaction.cs       |   78 +
 .../Impl/Transactions/Transaction.cs            |  155 +
 .../Impl/Transactions/TransactionImpl.cs        |  489 +++
 .../Impl/Transactions/TransactionMetricsImpl.cs |   62 +
 .../Impl/Transactions/TransactionsImpl.cs       |  201 ++
 .../Impl/Unmanaged/IUnmanagedTarget.cs          |   42 +
 .../Impl/Unmanaged/UnmanagedCallbackHandlers.cs |   99 +
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        | 1152 ++++++
 .../Impl/Unmanaged/UnmanagedContext.cs          |   53 +
 .../Unmanaged/UnmanagedNonReleaseableTarget.cs  |   68 +
 .../Impl/Unmanaged/UnmanagedTarget.cs           |   77 +
 .../Impl/Unmanaged/UnmanagedUtils.cs            | 1252 +++++++
 .../Lifecycle/ILifecycleBean.cs                 |   64 +
 .../Lifecycle/LifecycleEventType.cs             |   49 +
 .../Messaging/IMessageFilter.cs                 |   35 +
 .../Apache.Ignite.Core/Messaging/IMessaging.cs  |  105 +
 .../Portable/IPortableBuilder.cs                |   78 +
 .../Portable/IPortableIdMapper.cs               |   40 +
 .../Portable/IPortableMarshalAware.cs           |   39 +
 .../Portable/IPortableMetadata.cs               |   61 +
 .../Portable/IPortableNameMapper.cs             |   39 +
 .../Portable/IPortableObject.cs                 |   44 +
 .../Portable/IPortableRawReader.cs              |  264 ++
 .../Portable/IPortableRawWriter.cs              |  221 ++
 .../Portable/IPortableReader.cs                 |  340 ++
 .../Portable/IPortableSerializer.cs             |   39 +
 .../Portable/IPortableWriter.cs                 |  259 ++
 .../Apache.Ignite.Core/Portable/IPortables.cs   |  120 +
 .../Portable/PortableConfiguration.cs           |  122 +
 .../Portable/PortableException.cs               |   64 +
 .../Portable/PortableTypeConfiguration.cs       |  162 +
 .../Portable/PortableTypeNames.cs               |  115 +
 .../Resource/InstanceResourceAttribute.cs       |   35 +
 .../Resource/StoreSessionResourceAttribute.cs   |   32 +
 .../Apache.Ignite.Core/Services/IService.cs     |   51 +
 .../Services/IServiceContext.cs                 |   69 +
 .../Services/IServiceDescriptor.cs              |   96 +
 .../Apache.Ignite.Core/Services/IServices.cs    |  181 +
 .../Services/ServiceConfiguration.cs            |   62 +
 .../Services/ServiceInvocationException.cs      |  101 +
 .../Transactions/ITransaction.cs                |  230 ++
 .../Transactions/ITransactionMetrics.cs         |   47 +
 .../Transactions/ITransactions.cs               |   73 +
 .../Transactions/TransactionConcurrency.cs      |   36 +
 .../TransactionHeuristicException.cs            |   72 +
 .../Transactions/TransactionIsolation.cs        |   41 +
 .../TransactionOptimisticException.cs           |   69 +
 .../TransactionRollbackException.cs             |   68 +
 .../Transactions/TransactionState.cs            |   70 +
 .../Transactions/TransactionTimeoutException.cs |   69 +
 .../platform/src/main/dotnet/Apache.Ignite.sln  |   27 +-
 .../dotnet/PlatformDotNetConfiguration.java     |  119 -
 .../PlatformDotNetPortableConfiguration.java    |  228 --
 ...PlatformDotNetPortableTypeConfiguration.java |  248 --
 .../Apache.Ignite.Core.Tests.TestDll.csproj     |   52 +
 .../Properties/AssemblyInfo.cs                  |   49 +
 .../TestClass.cs                                |   35 +
 .../Apache.Ignite.Core.Tests.csproj             |  165 +-
 .../Cache/CacheAbstractTest.cs                  | 3321 ++++++++++++++++++
 .../Cache/CacheAffinityTest.cs                  |  139 +
 .../Cache/CacheDynamicStartTest.cs              |  281 ++
 .../Cache/CacheEntryTest.cs                     |   69 +
 .../Cache/CacheForkedTest.cs                    |   82 +
 .../Cache/CacheLocalAtomicTest.cs               |   57 +
 .../Cache/CacheLocalTest.cs                     |   56 +
 .../CachePartitionedAtomicNearEnabledTest.cs    |   50 +
 .../Cache/CachePartitionedAtomicTest.cs         |   50 +
 .../Cache/CachePartitionedNearEnabledTest.cs    |   50 +
 .../Cache/CachePartitionedTest.cs               |   50 +
 .../Cache/CacheReplicatedAtomicTest.cs          |   60 +
 .../Cache/CacheReplicatedTest.cs                |   60 +
 .../Cache/CacheTestAsyncWrapper.cs              |  436 +++
 .../Cache/Query/CacheQueriesTest.cs             |  928 +++++
 .../Continuous/ContinuousQueryAbstractTest.cs   | 1181 +++++++
 .../ContinuousQueryAtomicBackupTest.cs          |   33 +
 .../ContinuousQueryAtomicNoBackupTest.cs        |   34 +
 .../ContinuousQueryNoBackupAbstractTest.cs      |   72 +
 .../ContinuousQueryTransactionalBackupTest.cs   |   34 +
 .../ContinuousQueryTransactionalNoBackupTest.cs |   33 +
 .../Cache/Store/CacheParallelLoadStoreTest.cs   |  110 +
 .../Cache/Store/CacheStoreSessionTest.cs        |  285 ++
 .../Cache/Store/CacheStoreTest.cs               |  510 +++
 .../Cache/Store/CacheTestParallelLoadStore.cs   |   91 +
 .../Cache/Store/CacheTestStore.cs               |  155 +
 .../Compute/AbstractTaskTest.cs                 |  217 ++
 .../Compute/ClosureTaskTest.cs                  |  390 ++
 .../Compute/ComputeApiTest.cs                   | 1281 +++++++
 .../Compute/ComputeMultithreadedTest.cs         |  269 ++
 .../Compute/FailoverTaskSelfTest.cs             |  246 ++
 .../Forked/ForkedPortableClosureTaskTest.cs     |   33 +
 .../Compute/Forked/ForkedResourceTaskTest.cs    |   33 +
 .../Forked/ForkedSerializableClosureTaskTest.cs |   33 +
 .../Compute/Forked/ForkedTaskAdapterTest.cs     |   33 +
 .../Compute/IgniteExceptionTaskSelfTest.cs      |  753 ++++
 .../Compute/PortableClosureTaskTest.cs          |  217 ++
 .../Compute/PortableTaskTest.cs                 |  253 ++
 .../Compute/ResourceTaskTest.cs                 |  568 +++
 .../Compute/SerializableClosureTaskTest.cs      |  217 ++
 .../Compute/TaskAdapterTest.cs                  |  274 ++
 .../Compute/TaskResultTest.cs                   |  437 +++
 .../Config/Cache/Store/cache-store-session.xml  |   80 +
 .../Config/Compute/compute-grid1.xml            |   90 +
 .../Config/Compute/compute-grid2.xml            |   63 +
 .../Config/Compute/compute-grid3.xml            |   52 +
 .../Config/Compute/compute-standalone.xml       |   87 +
 .../Config/Dynamic/dynamic-client.xml           |   51 +
 .../Config/Dynamic/dynamic-data-no-cfg.xml      |   47 +
 .../Config/Dynamic/dynamic-data.xml             |   65 +
 .../Config/Ignite.exe.config.test               |   41 +
 .../Config/Lifecycle/lifecycle-beans.xml        |   66 +
 .../Config/Lifecycle/lifecycle-no-beans.xml     |   44 +
 .../Config/cache-portables.xml                  |   78 +
 .../Config/cache-query-continuous.xml           |  171 +
 .../Config/cache-query.xml                      |  100 +
 .../Config/marshaller-default.xml               |   43 +
 .../Config/marshaller-invalid.xml               |   46 +
 .../Config/marshaller-portable.xml              |   43 +
 .../native-client-test-cache-affinity.xml       |   70 +
 .../native-client-test-cache-parallel-store.xml |   69 +
 .../native-client-test-cache-portables.xml      |  226 ++
 .../Config/native-client-test-cache-store.xml   |  125 +
 .../Config/native-client-test-cache.xml         |  224 ++
 .../Config/portable.xml                         |   56 +
 .../Config/start-test-grid1.xml                 |   54 +
 .../Config/start-test-grid2.xml                 |   45 +
 .../Config/start-test-grid3.xml                 |   43 +
 .../Dataload/DataStreamerTest.cs                |  592 ++++
 .../Apache.Ignite.Core.Tests/EventsTest.cs      |  961 +++++
 .../Apache.Ignite.Core.Tests/ExceptionsTest.cs  |  352 ++
 .../Apache.Ignite.Core.Tests/ExecutableTest.cs  |  444 +++
 .../Apache.Ignite.Core.Tests/FutureTest.cs      |  278 ++
 .../IgniteManagerTest.cs                        |   51 +
 .../IgniteStartStopTest.cs                      |  380 ++
 .../Apache.Ignite.Core.Tests/IgnitionTest.cs    |   30 -
 .../Apache.Ignite.Core.Tests/LifecycleTest.cs   |  288 ++
 .../Apache.Ignite.Core.Tests/LoadDllTest.cs     |  243 ++
 .../Apache.Ignite.Core.Tests/MarshallerTest.cs  |   71 +
 .../Apache.Ignite.Core.Tests/MessagingTest.cs   |  646 ++++
 .../Portable/PortableApiSelfTest.cs             | 1787 ++++++++++
 .../Portable/PortableSelfTest.cs                | 2078 +++++++++++
 .../PortableConfigurationTest.cs                |  173 +
 .../Process/IIgniteProcessOutputReader.cs       |   35 +
 .../Process/IgniteProcess.cs                    |  292 ++
 .../Process/IgniteProcessConsoleOutputReader.cs |   40 +
 .../Query/ImplicitPortablePerson.cs             |   46 +
 .../Query/NoDefPortablePerson.cs                |   35 +
 .../Query/PortablePerson.cs                     |   69 +
 .../SerializationTest.cs                        |  240 ++
 .../Services/ServiceProxyTest.cs                |  741 ++++
 .../Services/ServicesAsyncWrapper.cs            |  174 +
 .../Services/ServicesTest.cs                    |  823 +++++
 .../Services/ServicesTestAsync.cs               |   33 +
 .../Apache.Ignite.Core.Tests/TestRunner.cs      |   15 +-
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |  292 ++
 .../TypeResolverTest.cs                         |  107 +
 .../platform/PlatformComputeBroadcastTask.java  |   73 +
 .../platform/PlatformComputeDecimalTask.java    |  106 +
 .../platform/PlatformComputeEchoTask.java       |  188 +
 .../ignite/platform/PlatformComputeEnum.java    |   28 +
 .../platform/PlatformComputeJavaPortable.java   |   39 +
 .../platform/PlatformComputePortable.java       |   42 +
 .../PlatformComputePortableArgTask.java         |  119 +
 .../platform/PlatformEventsWriteEventTask.java  |  146 +
 .../ignite/platform/PlatformMaxMemoryTask.java  |   57 +
 .../ignite/platform/PlatformMinMemoryTask.java  |   57 +
 .../lifecycle/PlatformJavaLifecycleBean.java    |   47 +
 .../lifecycle/PlatformJavaLifecycleTask.java    |   65 +
 parent/pom.xml                                  |    6 +
 389 files changed, 75307 insertions(+), 729 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
index 33a105b..52df2a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
@@ -61,6 +61,9 @@ import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.marshaller.MarshallerContext;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.platform.dotnet.PlatformDotNetConfiguration;
+import org.apache.ignite.platform.dotnet.PlatformDotNetPortableConfiguration;
+import org.apache.ignite.platform.dotnet.PlatformDotNetPortableTypeConfiguration;
 import org.apache.ignite.portable.PortableException;
 import org.apache.ignite.portable.PortableIdMapper;
 import org.apache.ignite.portable.PortableInvalidClassException;
@@ -239,6 +242,10 @@ public class PortableContext implements Externalizable {
 
         registerPredefinedType(PortableObjectImpl.class, 200);
         registerPredefinedType(PortableMetaDataImpl.class, 201);
+
+        registerPredefinedType(PlatformDotNetConfiguration.class, 202);
+        registerPredefinedType(PlatformDotNetPortableConfiguration.class, 203);
+        registerPredefinedType(PlatformDotNetPortableTypeConfiguration.class, 204);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetConfiguration.java b/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetConfiguration.java
new file mode 100644
index 0000000..80f4b26
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetConfiguration.java
@@ -0,0 +1,119 @@
+/*
+ * 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.platform.dotnet;
+
+import org.apache.ignite.configuration.PlatformConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.portable.PortableException;
+import org.apache.ignite.portable.PortableMarshalAware;
+import org.apache.ignite.portable.PortableRawReader;
+import org.apache.ignite.portable.PortableRawWriter;
+import org.apache.ignite.portable.PortableReader;
+import org.apache.ignite.portable.PortableWriter;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Mirror of .Net class Configuration.cs
+ */
+public class PlatformDotNetConfiguration implements PlatformConfiguration, PortableMarshalAware {
+    /** */
+    private PlatformDotNetPortableConfiguration portableCfg;
+
+    /** */
+    private List<String> assemblies;
+
+    /**
+     * Default constructor.
+     */
+    public PlatformDotNetConfiguration() {
+        // No-op.
+    }
+
+    /**
+     * Copy constructor.
+     *
+     * @param cfg Configuration to copy.
+     */
+    public PlatformDotNetConfiguration(PlatformDotNetConfiguration cfg) {
+        if (cfg.getPortableConfiguration() != null)
+            portableCfg = new PlatformDotNetPortableConfiguration(cfg.getPortableConfiguration());
+
+        if (cfg.getAssemblies() != null)
+            assemblies = new ArrayList<>(cfg.getAssemblies());
+    }
+
+    /**
+     * @return Configuration.
+     */
+    public PlatformDotNetPortableConfiguration getPortableConfiguration() {
+        return portableCfg;
+    }
+
+    /**
+     * @param portableCfg Configuration.
+     */
+    public void setPortableConfiguration(PlatformDotNetPortableConfiguration portableCfg) {
+        this.portableCfg = portableCfg;
+    }
+
+    /**
+     * @return Assemblies.
+     */
+    public List<String> getAssemblies() {
+        return assemblies;
+    }
+
+    /**
+     *
+     * @param assemblies Assemblies.
+     */
+    public void setAssemblies(List<String> assemblies) {
+        this.assemblies = assemblies;
+    }
+
+    /**
+     * @return Configuration copy.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    private PlatformDotNetConfiguration copy() {
+        return new PlatformDotNetConfiguration(this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writePortable(PortableWriter writer) throws PortableException {
+        PortableRawWriter rawWriter = writer.rawWriter();
+
+        rawWriter.writeObject(portableCfg);
+        rawWriter.writeCollection(assemblies);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readPortable(PortableReader reader) throws PortableException {
+        PortableRawReader rawReader = reader.rawReader();
+
+        portableCfg = rawReader.readObject();
+        assemblies = (List<String>)rawReader.<String>readCollection();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(PlatformDotNetConfiguration.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetPortableConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetPortableConfiguration.java b/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetPortableConfiguration.java
new file mode 100644
index 0000000..644a8e6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetPortableConfiguration.java
@@ -0,0 +1,228 @@
+/*
+ * 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.platform.dotnet;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.portable.PortableException;
+import org.apache.ignite.portable.PortableMarshalAware;
+import org.apache.ignite.portable.PortableRawReader;
+import org.apache.ignite.portable.PortableRawWriter;
+import org.apache.ignite.portable.PortableReader;
+import org.apache.ignite.portable.PortableWriter;
+
+import java.util.ArrayList;
+import java.util.Collection;
+
+/**
+ * Mirror of .Net class PortableConfiguration.cs
+ */
+public class PlatformDotNetPortableConfiguration implements PortableMarshalAware {
+    /** Type cfgs. */
+    private Collection<PlatformDotNetPortableTypeConfiguration> typesCfg;
+
+    /** Types. */
+    private Collection<String> types;
+
+    /** Default name mapper. */
+    private String dfltNameMapper;
+
+    /** Default id mapper. */
+    private String dfltIdMapper;
+
+    /** Default serializer. */
+    private String dfltSerializer;
+
+    /** Default metadata enabled. */
+    private boolean dfltMetadataEnabled = true;
+
+    /** Whether to cache deserialized value in IGridPortableObject */
+    private boolean dfltKeepDeserialized = true;
+
+    /**
+     * Default constructor.
+     */
+    public PlatformDotNetPortableConfiguration() {
+        // No-op.
+    }
+
+    /**
+     * Copy constructor.
+     * @param cfg configuration to copy.
+     */
+    public PlatformDotNetPortableConfiguration(PlatformDotNetPortableConfiguration cfg) {
+        if (cfg.getTypesConfiguration() != null) {
+            typesCfg = new ArrayList<>();
+
+            for (PlatformDotNetPortableTypeConfiguration typeCfg : cfg.getTypesConfiguration())
+                typesCfg.add(new PlatformDotNetPortableTypeConfiguration(typeCfg));
+        }
+
+        if (cfg.getTypes() != null)
+            types = new ArrayList<>(cfg.getTypes());
+
+        dfltNameMapper = cfg.getDefaultNameMapper();
+        dfltIdMapper = cfg.getDefaultIdMapper();
+        dfltSerializer = cfg.getDefaultSerializer();
+        dfltMetadataEnabled = cfg.isDefaultMetadataEnabled();
+        dfltKeepDeserialized = cfg.isDefaultKeepDeserialized();
+    }
+
+    /**
+     * @return Type cfgs.
+     */
+    public Collection<PlatformDotNetPortableTypeConfiguration> getTypesConfiguration() {
+        return typesCfg;
+    }
+
+    /**
+     * @param typesCfg New type cfgs.
+     */
+    public void setTypesConfiguration(Collection<PlatformDotNetPortableTypeConfiguration> typesCfg) {
+        this.typesCfg = typesCfg;
+    }
+
+    /**
+     * @return Types.
+     */
+    public Collection<String> getTypes() {
+        return types;
+    }
+
+    /**
+     * @param types New types.
+     */
+    public void setTypes(Collection<String> types) {
+        this.types = types;
+    }
+
+    /**
+     * @return Default name mapper.
+     */
+    public String getDefaultNameMapper() {
+        return dfltNameMapper;
+    }
+
+    /**
+     * @param dfltNameMapper New default name mapper.
+     */
+    public void setDefaultNameMapper(String dfltNameMapper) {
+        this.dfltNameMapper = dfltNameMapper;
+    }
+
+    /**
+     * @return Default id mapper.
+     */
+    public String getDefaultIdMapper() {
+        return dfltIdMapper;
+    }
+
+    /**
+     * @param dfltIdMapper New default id mapper.
+     */
+    public void setDefaultIdMapper(String dfltIdMapper) {
+        this.dfltIdMapper = dfltIdMapper;
+    }
+
+    /**
+     * @return Default serializer.
+     */
+    public String getDefaultSerializer() {
+        return dfltSerializer;
+    }
+
+    /**
+     * @param dfltSerializer New default serializer.
+     */
+    public void setDefaultSerializer(String dfltSerializer) {
+        this.dfltSerializer = dfltSerializer;
+    }
+
+    /**
+     * Gets default metadata enabled flag. See {@link #setDefaultMetadataEnabled(boolean)} for more information.
+     *
+     * @return Default metadata enabled flag.
+     */
+    public boolean isDefaultMetadataEnabled() {
+        return dfltMetadataEnabled;
+    }
+
+    /**
+     * Sets default metadata enabled flag. When set to {@code true} all portable types will save it's metadata to
+     * cluster.
+     * <p />
+     * Can be overridden for particular type using
+     * {@link PlatformDotNetPortableTypeConfiguration#setMetadataEnabled(Boolean)}.
+     *
+     * @param dfltMetadataEnabled Default metadata enabled flag.
+     */
+    public void setDefaultMetadataEnabled(boolean dfltMetadataEnabled) {
+        this.dfltMetadataEnabled = dfltMetadataEnabled;
+    }
+
+    /**
+     * Gets default keep deserialized flag. See {@link #setDefaultKeepDeserialized(boolean)} for more information.
+     *
+     * @return  Flag indicates whether to cache deserialized value in IGridPortableObject.
+     */
+    public boolean isDefaultKeepDeserialized() {
+        return dfltKeepDeserialized;
+    }
+
+    /**
+     * Sets default keep deserialized flag.
+     * <p />
+     * Can be overridden for particular type using
+     * {@link PlatformDotNetPortableTypeConfiguration#setKeepDeserialized(Boolean)}.
+     *
+     * @param keepDeserialized Keep deserialized flag.
+     */
+    public void setDefaultKeepDeserialized(boolean keepDeserialized) {
+        this.dfltKeepDeserialized = keepDeserialized;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writePortable(PortableWriter writer) throws PortableException {
+        PortableRawWriter rawWriter = writer.rawWriter();
+
+        rawWriter.writeCollection(typesCfg);
+        rawWriter.writeCollection(types);
+        rawWriter.writeString(dfltNameMapper);
+        rawWriter.writeString(dfltIdMapper);
+        rawWriter.writeString(dfltSerializer);
+        rawWriter.writeBoolean(dfltMetadataEnabled);
+        rawWriter.writeBoolean(dfltKeepDeserialized);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readPortable(PortableReader reader) throws PortableException {
+        PortableRawReader rawReader = reader.rawReader();
+
+        typesCfg = rawReader.readCollection();
+        types = rawReader.readCollection();
+        dfltNameMapper = rawReader.readString();
+        dfltIdMapper = rawReader.readString();
+        dfltSerializer = rawReader.readString();
+        dfltMetadataEnabled = rawReader.readBoolean();
+        dfltKeepDeserialized = rawReader.readBoolean();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(PlatformDotNetPortableConfiguration.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetPortableTypeConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetPortableTypeConfiguration.java b/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetPortableTypeConfiguration.java
new file mode 100644
index 0000000..b6fdbde
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetPortableTypeConfiguration.java
@@ -0,0 +1,248 @@
+/*
+ * 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.platform.dotnet;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.portable.PortableException;
+import org.apache.ignite.portable.PortableMarshalAware;
+import org.apache.ignite.portable.PortableRawReader;
+import org.apache.ignite.portable.PortableRawWriter;
+import org.apache.ignite.portable.PortableReader;
+import org.apache.ignite.portable.PortableWriter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Mirror of .Net class GridDotNetPortableTypeConfiguration.cs
+ */
+public class PlatformDotNetPortableTypeConfiguration implements PortableMarshalAware {
+    /** Assembly name. */
+    private String assemblyName;
+
+    /** Type name. */
+    private String typeName;
+
+    /** Name mapper. */
+    private String nameMapper;
+
+    /** Id mapper. */
+    private String idMapper;
+
+    /** Serializer. */
+    private String serializer;
+
+    /** Affinity key field name. */
+    private String affinityKeyFieldName;
+
+    /** Metadata enabled. */
+    private Boolean metadataEnabled;
+
+    /** Whether to cache deserialized value in IGridPortableObject. */
+    private Boolean keepDeserialized;
+
+    /**
+     * Default constructor.
+     */
+    public PlatformDotNetPortableTypeConfiguration() {
+        // No-op.
+    }
+
+    /**
+     * Copy constructor.
+     * @param cfg configuration to copy.
+     */
+    public PlatformDotNetPortableTypeConfiguration(PlatformDotNetPortableTypeConfiguration cfg) {
+        assemblyName = cfg.getAssemblyName();
+        typeName = cfg.getTypeName();
+        nameMapper = cfg.getNameMapper();
+        idMapper = cfg.getIdMapper();
+        serializer = cfg.getSerializer();
+        affinityKeyFieldName = cfg.getAffinityKeyFieldName();
+        metadataEnabled = cfg.getMetadataEnabled();
+        keepDeserialized = cfg.isKeepDeserialized();
+    }
+
+    /**
+     * @return Assembly name.
+     */
+    public String getAssemblyName() {
+        return assemblyName;
+    }
+
+    /**
+     * @param assemblyName New assembly name.
+     */
+    public void setAssemblyName(String assemblyName) {
+        this.assemblyName = assemblyName;
+    }
+
+    /**
+     * @return Type name.
+     */
+    public String getTypeName() {
+        return typeName;
+    }
+
+    /**
+     * @param typeName New type name.
+     */
+    public void setTypeName(String typeName) {
+        this.typeName = typeName;
+    }
+
+    /**
+     * @return Name mapper.
+     */
+    public String getNameMapper() {
+        return nameMapper;
+    }
+
+    /**
+     * @param nameMapper New name mapper.
+     */
+    public void setNameMapper(String nameMapper) {
+        this.nameMapper = nameMapper;
+    }
+
+    /**
+     * @return Id mapper.
+     */
+    public String getIdMapper() {
+        return idMapper;
+    }
+
+    /**
+     * @param idMapper New id mapper.
+     */
+    public void setIdMapper(String idMapper) {
+        this.idMapper = idMapper;
+    }
+
+    /**
+     * @return Serializer.
+     */
+    public String getSerializer() {
+        return serializer;
+    }
+
+    /**
+     * @param serializer New serializer.
+     */
+    public void setSerializer(String serializer) {
+        this.serializer = serializer;
+    }
+
+    /**
+     * Gets metadata enabled flag. See {@link #setMetadataEnabled(Boolean)} for more information.
+     *
+     * @return Metadata enabled flag.
+     */
+    public Boolean getMetadataEnabled() {
+        return metadataEnabled;
+    }
+
+    /**
+     * Sets metadata enabled flag.
+     * <p />
+     * When set to {@code null} default value taken from
+     * {@link PlatformDotNetPortableConfiguration#isDefaultMetadataEnabled()} will be used.
+     *
+     * @param metadataEnabled New metadata enabled.
+     */
+    public void setMetadataEnabled(Boolean metadataEnabled) {
+        this.metadataEnabled = metadataEnabled;
+    }
+
+    /**
+     * @return Affinity key field name.
+     */
+    public String getAffinityKeyFieldName() {
+        return affinityKeyFieldName;
+    }
+
+    /**
+     * @param affinityKeyFieldName Affinity key field name.
+     */
+    public void setAffinityKeyFieldName(String affinityKeyFieldName) {
+        this.affinityKeyFieldName = affinityKeyFieldName;
+    }
+
+    /**
+     * Gets keep deserialized flag.
+     *
+     * @return Flag indicates whether to cache deserialized value in IGridPortableObject.
+     * @deprecated Use {@link #getKeepDeserialized()} instead.
+     */
+    @Deprecated
+    @Nullable public Boolean isKeepDeserialized() {
+        return keepDeserialized;
+    }
+
+    /**
+     * Gets keep deserialized flag. See {@link #setKeepDeserialized(Boolean)} for more information.
+     *
+     * @return Flag indicates whether to cache deserialized value in IGridPortableObject.
+     */
+    @Nullable public Boolean getKeepDeserialized() {
+        return keepDeserialized;
+    }
+
+    /**
+     * Sets keep deserialized flag.
+     * <p />
+     * When set to {@code null} default value taken from
+     * {@link PlatformDotNetPortableConfiguration#isDefaultKeepDeserialized()} will be used.
+     *
+     * @param keepDeserialized Keep deserialized flag.
+     */
+    public void setKeepDeserialized(@Nullable Boolean keepDeserialized) {
+        this.keepDeserialized = keepDeserialized;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writePortable(PortableWriter writer) throws PortableException {
+        PortableRawWriter rawWriter = writer.rawWriter();
+
+        rawWriter.writeString(assemblyName);
+        rawWriter.writeString(typeName);
+        rawWriter.writeString(nameMapper);
+        rawWriter.writeString(idMapper);
+        rawWriter.writeString(serializer);
+        rawWriter.writeString(affinityKeyFieldName);
+        rawWriter.writeObject(metadataEnabled);
+        rawWriter.writeObject(keepDeserialized);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readPortable(PortableReader reader) throws PortableException {
+        PortableRawReader rawReader = reader.rawReader();
+
+        assemblyName = rawReader.readString();
+        typeName = rawReader.readString();
+        nameMapper = rawReader.readString();
+        idMapper = rawReader.readString();
+        serializer = rawReader.readString();
+        affinityKeyFieldName = rawReader.readString();
+        metadataEnabled = rawReader.readObject();
+        keepDeserialized = rawReader.readObject();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(PlatformDotNetPortableTypeConfiguration.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/pom.xml
----------------------------------------------------------------------
diff --git a/modules/platform/pom.xml b/modules/platform/pom.xml
index 56e1821..ac34436 100644
--- a/modules/platform/pom.xml
+++ b/modules/platform/pom.xml
@@ -63,4 +63,18 @@
         </dependency>
     </dependencies>
 
+    <build>
+        <resources>
+            <resource>
+                <directory>src/main/java</directory>
+                <excludes>
+                    <exclude>**/*.java</exclude>
+                </excludes>
+            </resource>
+
+            <resource>
+                <directory>src/main/resources</directory>
+            </resource>
+        </resources>
+    </build>
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index 12e335a..52f9b48 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -49,11 +49,121 @@
     <Reference Include="System.Core" />
   </ItemGroup>
   <ItemGroup>
+    <Compile Include="Cache\CacheAtomicUpdateTimeoutException.cs" />
+    <Compile Include="Cache\CacheEntryProcessorException.cs" />
+    <Compile Include="Cache\CacheException.cs" />
+    <Compile Include="Cache\CachePartialUpdateException.cs" />
+    <Compile Include="Cache\CachePeekMode.cs" />
+    <Compile Include="Cache\Event\CacheEntryEventType.cs" />
+    <Compile Include="Cache\Event\ICacheEntryEvent.cs" />
+    <Compile Include="Cache\Event\ICacheEntryEventFilter.cs" />
+    <Compile Include="Cache\Event\ICacheEntryEventListener.cs" />
+    <Compile Include="Cache\Expiry\ExpiryPolicy.cs" />
+    <Compile Include="Cache\Expiry\IExpiryPolicy.cs" />
+    <Compile Include="Cache\ICache.cs" />
+    <Compile Include="Cache\ICacheAffinity.cs" />
+    <Compile Include="Cache\ICacheEntry.cs" />
+    <Compile Include="Cache\ICacheEntryFilter.cs" />
+    <Compile Include="Cache\ICacheEntryProcessor.cs" />
+    <Compile Include="Cache\ICacheEntryProcessorResult.cs" />
+    <Compile Include="Cache\ICacheLock.cs" />
+    <Compile Include="Cache\ICacheMetrics.cs" />
+    <Compile Include="Cache\IMutableCacheEntry.cs" />
+    <Compile Include="Cache\Query\Continuous\ContinuousQuery.cs" />
+    <Compile Include="Cache\Query\Continuous\IContinuousQueryHandle.cs" />
+    <Compile Include="Cache\Query\IQueryCursor.cs" />
+    <Compile Include="Cache\Query\QueryBase.cs" />
+    <Compile Include="Cache\Query\ScanQuery.cs" />
+    <Compile Include="Cache\Query\SqlFieldsQuery.cs" />
+    <Compile Include="Cache\Query\SqlQuery.cs" />
+    <Compile Include="Cache\Query\TextQuery.cs" />
+    <Compile Include="Cache\Store\CacheParallelLoadStoreAdapter.cs" />
+    <Compile Include="Cache\Store\CacheStoreAdapter.cs" />
+    <Compile Include="Cache\Store\CacheStoreException.cs" />
+    <Compile Include="Cache\Store\ICacheStore.cs" />
+    <Compile Include="Cache\Store\ICacheStoreSession.cs" />
+    <Compile Include="Cluster\ClusterGroupEmptyException.cs" />
+    <Compile Include="Cluster\ClusterTopologyException.cs" />
+    <Compile Include="Cluster\ICluster.cs" />
+    <Compile Include="Cluster\IClusterGroup.cs" />
+    <Compile Include="Cluster\IClusterMetrics.cs" />
+    <Compile Include="Cluster\IClusterNode.cs" />
+    <Compile Include="Cluster\IClusterNodeFilter.cs" />
     <Compile Include="Common\IgniteException.cs" />
     <Compile Include="Common\IAsyncSupport.cs" />
     <Compile Include="Common\IFuture.cs" />
+    <Compile Include="Common\IgniteGuid.cs" />
+    <Compile Include="Compute\ComputeExecutionRejectedException.cs" />
+    <Compile Include="Compute\ComputeJobAdapter.cs" />
+    <Compile Include="Compute\ComputeJobFailoverException.cs" />
+    <Compile Include="Compute\ComputeJobResultPolicy.cs" />
+    <Compile Include="Compute\ComputeTaskAdapter.cs" />
+    <Compile Include="Compute\ComputeTaskCancelledException.cs" />
+    <Compile Include="Compute\ComputeTaskNoResultCacheAttribute.cs" />
+    <Compile Include="Compute\ComputeTaskSplitAdapter.cs" />
+    <Compile Include="Compute\ComputeTaskTimeoutException.cs" />
+    <Compile Include="Compute\ComputeUserUndeclaredException.cs" />
+    <Compile Include="Compute\ICompute.cs" />
+    <Compile Include="Compute\IComputeFunc.cs" />
+    <Compile Include="Compute\IComputeJob.cs" />
+    <Compile Include="Compute\IComputeJobResult.cs" />
+    <Compile Include="Compute\IComputeReducer.cs" />
+    <Compile Include="Compute\IComputeTask.cs" />
+    <Compile Include="Datastream\IDataStreamer.cs" />
+    <Compile Include="Datastream\IStreamReceiver.cs" />
+    <Compile Include="Datastream\StreamTransformer.cs" />
+    <Compile Include="Datastream\StreamVisitor.cs" />
+    <Compile Include="Events\CacheEvent.cs" />
+    <Compile Include="Events\CacheQueryExecutedEvent.cs" />
+    <Compile Include="Events\CacheQueryReadEvent.cs" />
+    <Compile Include="Events\CacheRebalancingEvent.cs" />
+    <Compile Include="Events\CheckpointEvent.cs" />
+    <Compile Include="Events\DiscoveryEvent.cs" />
+    <Compile Include="Events\EventBase.cs" />
+    <Compile Include="Events\EventReader.cs" />
+    <Compile Include="Events\EventType.cs" />
+    <Compile Include="Events\IEvent.cs" />
+    <Compile Include="Events\IEventFilter.cs" />
+    <Compile Include="Events\IEvents.cs" />
+    <Compile Include="Events\JobEvent.cs" />
+    <Compile Include="Events\SwapSpaceEvent.cs" />
+    <Compile Include="Events\TaskEvent.cs" />
+    <Compile Include="IgniteConfiguration.cs" />
     <Compile Include="Ignition.cs" />
     <Compile Include="Common\AsyncSupportedAttribute.cs" />
+    <Compile Include="IIgnite.cs" />
+    <Compile Include="Impl\Cache\CacheAffinityImpl.cs" />
+    <Compile Include="Impl\Cache\CacheEntry.cs" />
+    <Compile Include="Impl\Cache\CacheEntryFilterHolder.cs" />
+    <Compile Include="Impl\Cache\CacheEntryProcessorHolder.cs" />
+    <Compile Include="Impl\Cache\CacheEntryProcessorResult.cs" />
+    <Compile Include="Impl\Cache\CacheEntryProcessorResultHolder.cs" />
+    <Compile Include="Impl\Cache\CacheEnumerable.cs" />
+    <Compile Include="Impl\Cache\CacheEnumerator.cs" />
+    <Compile Include="Impl\Cache\CacheEnumeratorProxy.cs" />
+    <Compile Include="Impl\Cache\CacheImpl.cs" />
+    <Compile Include="Impl\Cache\CacheLock.cs" />
+    <Compile Include="Impl\Cache\CacheMetricsImpl.cs" />
+    <Compile Include="Impl\Cache\CacheOp.cs" />
+    <Compile Include="Impl\Cache\CacheProxyImpl.cs" />
+    <Compile Include="Impl\Cache\Event\CacheEntryCreateEvent.cs" />
+    <Compile Include="Impl\Cache\Event\CacheEntryRemoveEvent.cs" />
+    <Compile Include="Impl\Cache\Event\CacheEntryUpdateEvent.cs" />
+    <Compile Include="Impl\Cache\MutableCacheEntry.cs" />
+    <Compile Include="Impl\Cache\Query\AbstractQueryCursor.cs" />
+    <Compile Include="Impl\Cache\Query\Continuous\ContinuousQueryFilter.cs" />
+    <Compile Include="Impl\Cache\Query\Continuous\ContinuousQueryFilterHolder.cs" />
+    <Compile Include="Impl\Cache\Query\Continuous\ContinuousQueryHandleImpl.cs" />
+    <Compile Include="Impl\Cache\Query\Continuous\ContinuousQueryUtils.cs" />
+    <Compile Include="Impl\Cache\Query\FieldsQueryCursor.cs" />
+    <Compile Include="Impl\Cache\Query\QueryCursor.cs" />
+    <Compile Include="Impl\Cache\Store\CacheStore.cs" />
+    <Compile Include="Impl\Cache\Store\CacheStoreSession.cs" />
+    <Compile Include="Impl\Cache\Store\CacheStoreSessionProxy.cs" />
+    <Compile Include="Impl\Cluster\ClusterGroupImpl.cs" />
+    <Compile Include="Impl\Cluster\ClusterMetricsImpl.cs" />
+    <Compile Include="Impl\Cluster\ClusterNodeImpl.cs" />
+    <Compile Include="Impl\Cluster\IClusterGroupEx.cs" />
     <Compile Include="Impl\Collections\CollectionExtensions.cs" />
     <Compile Include="Impl\Collections\MultiValueDictionary.cs" />
     <Compile Include="Impl\Collections\ReadOnlyCollection.cs" />
@@ -62,16 +172,60 @@
     <Compile Include="Impl\Common\CompletedAsyncResult.cs" />
     <Compile Include="Impl\Common\CopyOnWriteConcurrentDictionary.cs" />
     <Compile Include="Impl\Common\DelegateConverter.cs" />
+    <Compile Include="Impl\Common\DelegateTypeDescriptor.cs" />
     <Compile Include="Impl\Common\Future.cs" />
+    <Compile Include="Impl\Common\FutureConverter.cs" />
     <Compile Include="Impl\Common\FutureType.cs" />
-    <Compile Include="Impl\Common\GridArgumentCheck.cs" />
+    <Compile Include="Impl\Common\IgniteArgumentCheck.cs" />
     <Compile Include="Impl\Common\IFutureConverter.cs" />
     <Compile Include="Impl\Common\IFutureInternal.cs" />
     <Compile Include="Impl\Common\LoadedAssembliesResolver.cs" />
+    <Compile Include="Impl\Common\PortableResultWrapper.cs" />
     <Compile Include="Impl\Common\TypeCaster.cs" />
+    <Compile Include="Impl\Compute\Closure\ComputeAbstractClosureTask.cs" />
+    <Compile Include="Impl\Compute\Closure\ComputeActionJob.cs" />
+    <Compile Include="Impl\Compute\Closure\ComputeFuncJob.cs" />
+    <Compile Include="Impl\Compute\Closure\ComputeMultiClosureTask.cs" />
+    <Compile Include="Impl\Compute\Closure\ComputeOutFuncJob.cs" />
+    <Compile Include="Impl\Compute\Closure\ComputeReducingClosureTask.cs" />
+    <Compile Include="Impl\Compute\Closure\ComputeSingleClosureTask.cs" />
+    <Compile Include="Impl\Compute\Closure\IComputeResourceInjector.cs" />
+    <Compile Include="Impl\Compute\Compute.cs" />
+    <Compile Include="Impl\Compute\ComputeAsync.cs" />
+    <Compile Include="Impl\Compute\ComputeFunc.cs" />
+    <Compile Include="Impl\Compute\ComputeImpl.cs" />
+    <Compile Include="Impl\Compute\ComputeJob.cs" />
+    <Compile Include="Impl\Compute\ComputeJobHolder.cs" />
+    <Compile Include="Impl\Compute\ComputeJobResultGenericWrapper.cs" />
+    <Compile Include="Impl\Compute\ComputeJobResultImpl.cs" />
+    <Compile Include="Impl\Compute\ComputeOutFunc.cs" />
+    <Compile Include="Impl\Compute\ComputeTaskHolder.cs" />
+    <Compile Include="Impl\Datastream\DataStreamerBatch.cs" />
+    <Compile Include="Impl\Datastream\DataStreamerEntry.cs" />
+    <Compile Include="Impl\Datastream\DataStreamerImpl.cs" />
+    <Compile Include="Impl\Datastream\DataStreamerRemoveEntry.cs" />
+    <Compile Include="Impl\Datastream\StreamReceiverHolder.cs" />
+    <Compile Include="Impl\Events\Events.cs" />
+    <Compile Include="Impl\Events\EventsAsync.cs" />
+    <Compile Include="Impl\Events\RemoteListenEventFilter.cs" />
+    <Compile Include="Impl\ExceptionUtils.cs" />
+    <Compile Include="Impl\IgniteConfigurationEx.cs" />
+    <Compile Include="Impl\Ignite.cs" />
+    <Compile Include="Impl\IgniteManager.cs" />
+    <Compile Include="Impl\IgniteProxy.cs" />
+    <Compile Include="Impl\PlatformTarget.cs" />
+    <Compile Include="Impl\IgniteUtils.cs" />
     <Compile Include="Impl\Handle\Handle.cs" />
     <Compile Include="Impl\Handle\HandleRegistry.cs" />
     <Compile Include="Impl\Handle\IHandle.cs" />
+    <Compile Include="Impl\IInteropCallback.cs" />
+    <Compile Include="Impl\InteropExceptionHolder.cs" />
+    <Compile Include="Impl\Interop\InteropDotNetConfiguration.cs" />
+    <Compile Include="Impl\Interop\InteropDotNetPortableConfiguration.cs" />
+    <Compile Include="Impl\Interop\InteropDotNetPortableTypeConfiguration.cs" />
+    <Compile Include="Impl\LifecycleBeanHolder.cs" />
+    <Compile Include="Impl\Memory\InteropExternalMemory.cs" />
+    <Compile Include="Impl\Memory\InteropMemoryUtils.cs" />
     <Compile Include="Impl\Memory\IPlatformMemory.cs" />
     <Compile Include="Impl\Memory\PlatformBigEndianMemoryStream.cs" />
     <Compile Include="Impl\Memory\PlatformMemory.cs" />
@@ -82,10 +236,135 @@
     <Compile Include="Impl\Memory\PlatformPooledMemory.cs" />
     <Compile Include="Impl\Memory\PlatformRawMemory.cs" />
     <Compile Include="Impl\Memory\PlatformUnpooledMemory.cs" />
+    <Compile Include="Impl\Messaging\MessageFilterHolder.cs" />
+    <Compile Include="Impl\Messaging\Messaging.cs" />
+    <Compile Include="Impl\Messaging\MessagingAsync.cs" />
+    <Compile Include="Impl\NativeMethods.cs" />
     <Compile Include="Impl\Portable\IO\IPortableStream.cs" />
+    <Compile Include="Impl\Portable\IO\PortableAbstractStream.cs" />
+    <Compile Include="Impl\Portable\IO\PortableHeapStream.cs" />
+    <Compile Include="Impl\Portable\IO\PortableStreamAdapter.cs" />
+    <Compile Include="Impl\Portable\IPortableSystemTypeSerializer.cs" />
+    <Compile Include="Impl\Portable\IPortableTypeDescriptor.cs" />
+    <Compile Include="Impl\Portable\IPortableWriteAware.cs" />
+    <Compile Include="Impl\Portable\Metadata\IPortableMetadataHandler.cs" />
+    <Compile Include="Impl\Portable\Metadata\PortableHashsetMetadataHandler.cs" />
+    <Compile Include="Impl\Portable\Metadata\PortableMetadataHolder.cs" />
+    <Compile Include="Impl\Portable\Metadata\PortableMetadataImpl.cs" />
+    <Compile Include="Impl\Portable\PortableBuilderField.cs" />
+    <Compile Include="Impl\Portable\PortableBuilderImpl.cs" />
+    <Compile Include="Impl\Portable\PortableCollectionInfo.cs" />
+    <Compile Include="Impl\Portable\PortableFullTypeDescriptor.cs" />
+    <Compile Include="Impl\Portable\PortableHandleDictionary.cs" />
+    <Compile Include="Impl\Portable\PortableMarshalAwareSerializer.cs" />
+    <Compile Include="Impl\Portable\PortableMarshaller.cs" />
+    <Compile Include="Impl\Portable\PortableMode.cs" />
+    <Compile Include="Impl\Portable\PortableObjectHandle.cs" />
+    <Compile Include="Impl\Portable\PortableOrSerializableObjectHolder.cs" />
+    <Compile Include="Impl\Portable\PortableReaderHandleDictionary.cs" />
+    <Compile Include="Impl\Portable\PortableReaderImpl.cs" />
+    <Compile Include="Impl\Portable\PortableReflectiveRoutines.cs" />
+    <Compile Include="Impl\Portable\PortableReflectiveSerializer.cs" />
+    <Compile Include="Impl\Portable\PortablesImpl.cs" />
+    <Compile Include="Impl\Portable\PortableSurrogateTypeDescriptor.cs" />
+    <Compile Include="Impl\Portable\PortableSystemHandlers.cs" />
+    <Compile Include="Impl\Portable\PortableSystemTypeSerializer.cs" />
+    <Compile Include="Impl\Portable\PortableUserObject.cs" />
+    <Compile Include="Impl\Portable\PortableUtils.cs" />
+    <Compile Include="Impl\Portable\PortableWriterImpl.cs" />
+    <Compile Include="Impl\Portable\SerializableObjectHolder.cs" />
+    <Compile Include="Impl\Portable\TypeResolver.cs" />
+    <Compile Include="Impl\Resource\IResourceInjector.cs" />
+    <Compile Include="Impl\Resource\ResourceFieldInjector.cs" />
+    <Compile Include="Impl\Resource\ResourceMethodInjector.cs" />
+    <Compile Include="Impl\Resource\ResourceProcessor.cs" />
+    <Compile Include="Impl\Resource\ResourcePropertyInjector.cs" />
+    <Compile Include="Impl\Resource\ResourceTypeDescriptor.cs" />
+    <Compile Include="Impl\Services\ServiceContext.cs" />
+    <Compile Include="Impl\Services\ServiceDescriptor.cs" />
+    <Compile Include="Impl\Services\ServiceProxy.cs" />
+    <Compile Include="Impl\Services\ServiceProxyInvoker.cs" />
+    <Compile Include="Impl\Services\ServiceProxySerializer.cs" />
+    <Compile Include="Impl\Services\Services.cs" />
+    <Compile Include="Impl\Services\ServicesAsync.cs" />
+    <Compile Include="Impl\Transactions\AsyncTransaction.cs" />
+    <Compile Include="Impl\Transactions\Transaction.cs" />
+    <Compile Include="Impl\Transactions\TransactionImpl.cs" />
+    <Compile Include="Impl\Transactions\TransactionMetricsImpl.cs" />
+    <Compile Include="Impl\Transactions\TransactionsImpl.cs" />
+    <Compile Include="Impl\Unmanaged\IUnmanagedTarget.cs" />
+    <Compile Include="Impl\Unmanaged\UnmanagedCallbackHandlers.cs" />
+    <Compile Include="Impl\Unmanaged\UnmanagedCallbacks.cs" />
+    <Compile Include="Impl\Unmanaged\UnmanagedContext.cs" />
+    <Compile Include="Impl\Unmanaged\UnmanagedNonReleaseableTarget.cs" />
+    <Compile Include="Impl\Unmanaged\UnmanagedTarget.cs" />
+    <Compile Include="Impl\Unmanaged\UnmanagedUtils.cs" />
+    <Compile Include="Lifecycle\ILifecycleBean.cs" />
+    <Compile Include="Lifecycle\LifecycleEventType.cs" />
+    <Compile Include="Messaging\IMessageFilter.cs" />
+    <Compile Include="Messaging\IMessaging.cs" />
+    <Compile Include="Portable\IPortableBuilder.cs" />
+    <Compile Include="Portable\IPortableIdMapper.cs" />
+    <Compile Include="Portable\IPortableMarshalAware.cs" />
+    <Compile Include="Portable\IPortableMetadata.cs" />
+    <Compile Include="Portable\IPortableNameMapper.cs" />
+    <Compile Include="Portable\IPortableObject.cs" />
+    <Compile Include="Portable\IPortableRawReader.cs" />
+    <Compile Include="Portable\IPortableRawWriter.cs" />
+    <Compile Include="Portable\IPortableReader.cs" />
+    <Compile Include="Portable\IPortables.cs" />
+    <Compile Include="Portable\IPortableSerializer.cs" />
+    <Compile Include="Portable\IPortableWriter.cs" />
+    <Compile Include="Portable\PortableConfiguration.cs" />
+    <Compile Include="Portable\PortableException.cs" />
+    <Compile Include="Portable\PortableTypeConfiguration.cs" />
+    <Compile Include="Portable\PortableTypeNames.cs" />
     <Compile Include="Properties\AssemblyInfo.cs" />
+    <Compile Include="Resource\InstanceResourceAttribute.cs" />
+    <Compile Include="Resource\StoreSessionResourceAttribute.cs" />
+    <Compile Include="Services\IService.cs" />
+    <Compile Include="Services\IServiceContext.cs" />
+    <Compile Include="Services\IServiceDescriptor.cs" />
+    <Compile Include="Services\IServices.cs" />
+    <Compile Include="Services\ServiceConfiguration.cs" />
+    <Compile Include="Services\ServiceInvocationException.cs" />
+    <Compile Include="Transactions\ITransaction.cs" />
+    <Compile Include="Transactions\ITransactionMetrics.cs" />
+    <Compile Include="Transactions\ITransactions.cs" />
+    <Compile Include="Transactions\TransactionConcurrency.cs" />
+    <Compile Include="Transactions\TransactionHeuristicException.cs" />
+    <Compile Include="Transactions\TransactionIsolation.cs" />
+    <Compile Include="Transactions\TransactionOptimisticException.cs" />
+    <Compile Include="Transactions\TransactionRollbackException.cs" />
+    <Compile Include="Transactions\TransactionState.cs" />
+    <Compile Include="Transactions\TransactionTimeoutException.cs" />
+  </ItemGroup>
+  <ItemGroup>
+    <ProjectReference Include="..\..\cpp\common\project\vs\common.vcxproj">
+      <Project>{4f7e4917-4612-4b96-9838-025711ade391}</Project>
+      <Name>common</Name>
+    </ProjectReference>
+  </ItemGroup>
+  <ItemGroup Condition="'$(Configuration)|$(Platform)' == 'Debug|x64'">
+    <EmbeddedResource Include="..\x64\Debug\ignite.common.dll">
+      <Link>resources\debug\x64\ignite.common.dll</Link>
+    </EmbeddedResource>
+  </ItemGroup>
+  <ItemGroup Condition="'$(Configuration)|$(Platform)' == 'Release|x64'">
+    <EmbeddedResource Include="..\x64\Release\ignite.common.dll">
+      <Link>resources\release\x64\ignite.common.dll</Link>
+    </EmbeddedResource>
+  </ItemGroup>
+  <ItemGroup Condition="'$(Configuration)|$(Platform)' == 'Debug|x86'">
+    <EmbeddedResource Include="..\Win32\Debug\ignite.common.dll">
+      <Link>resources\debug\x86\ignite.common.dll</Link>
+    </EmbeddedResource>
+  </ItemGroup>
+  <ItemGroup Condition="'$(Configuration)|$(Platform)' == 'Release|x86'">
+    <EmbeddedResource Include="..\Win32\Release\ignite.common.dll">
+      <Link>resources\release\x86\ignite.common.dll</Link>
+    </EmbeddedResource>
   </ItemGroup>
-  <ItemGroup />
   <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />
   <!-- To modify your build process, add your task inside one of the targets below and uncomment it. 
        Other similar extension points exist, see Microsoft.Common.targets.

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CacheAtomicUpdateTimeoutException.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CacheAtomicUpdateTimeoutException.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CacheAtomicUpdateTimeoutException.cs
new file mode 100644
index 0000000..f0b5987
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CacheAtomicUpdateTimeoutException.cs
@@ -0,0 +1,67 @@
+/*
+ * 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.Cache
+{
+    using System;
+    using System.Runtime.Serialization;
+
+    /// <summary>
+    /// Indicates atomic operation timeout.
+    /// </summary>
+    [Serializable]
+    public class CacheAtomicUpdateTimeoutException : CacheException
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheAtomicUpdateTimeoutException"/> class.
+        /// </summary>
+        public CacheAtomicUpdateTimeoutException()
+        {
+            // No-op.
+        }
+
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheAtomicUpdateTimeoutException"/> class.
+        /// </summary>
+        /// <param name="message">The message that describes the error.</param>
+        public CacheAtomicUpdateTimeoutException(string message) : base(message)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheAtomicUpdateTimeoutException"/> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        public CacheAtomicUpdateTimeoutException(string message, Exception cause) : base(message, cause)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheAtomicUpdateTimeoutException"/> class.
+        /// </summary>
+        /// <param name="info">Serialization information.</param>
+        /// <param name="ctx">Streaming context.</param>
+        protected CacheAtomicUpdateTimeoutException(SerializationInfo info, StreamingContext ctx) : base(info, ctx)
+        {
+            // No-op.
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CacheEntryProcessorException.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CacheEntryProcessorException.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CacheEntryProcessorException.cs
new file mode 100644
index 0000000..341c713
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CacheEntryProcessorException.cs
@@ -0,0 +1,79 @@
+/*
+ * 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.Cache
+{
+    using System;
+    using System.Runtime.Serialization;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// An exception to indicate a problem occurred attempting to execute an 
+    /// <see cref="ICacheEntryProcessor{K, V, A, R}"/> against an entry.
+    /// </summary>
+    [Serializable]
+    public class CacheEntryProcessorException : IgniteException
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheEntryProcessorException"/> class.
+        /// </summary>
+        public CacheEntryProcessorException()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheEntryProcessorException"/> class.
+        /// </summary>
+        /// <param name="message">The message that describes the error.</param>
+        public CacheEntryProcessorException(string message) : base(message)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheEntryProcessorException"/> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        public CacheEntryProcessorException(string message, Exception cause)
+            : base(message, cause)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheEntryProcessorException"/> class.
+        /// </summary>
+        /// <param name="innerException">The inner exception.</param>
+        public CacheEntryProcessorException(Exception innerException)
+            : base("Error occurred in CacheEntryProcessor, see InnerException for details.", innerException)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheEntryProcessorException"/> class.
+        /// </summary>
+        /// <param name="info">Serialization information.</param>
+        /// <param name="ctx">Streaming context.</param>
+        protected CacheEntryProcessorException(SerializationInfo info, StreamingContext ctx) : base(info, ctx)
+        {
+            // No-op.
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CacheException.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CacheException.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CacheException.cs
new file mode 100644
index 0000000..c00f115
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CacheException.cs
@@ -0,0 +1,68 @@
+/*
+ * 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.Cache
+{
+    using System;
+    using System.Runtime.Serialization;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// Indicates an error during Cache operation.
+    /// </summary>
+    [Serializable]
+    public class CacheException : IgniteException
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheException"/> class.
+        /// </summary>
+        public CacheException()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheException"/> class.
+        /// </summary>
+        /// <param name="message">The message that describes the error.</param>
+        public CacheException(string message) : base(message)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheException"/> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        public CacheException(string message, Exception cause) : base(message, cause)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheException"/> class.
+        /// </summary>
+        /// <param name="info">Serialization information.</param>
+        /// <param name="ctx">Streaming context.</param>
+        protected CacheException(SerializationInfo info, StreamingContext ctx)
+            : base(info, ctx)
+        {
+            // No-op.
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CachePartialUpdateException.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CachePartialUpdateException.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CachePartialUpdateException.cs
new file mode 100644
index 0000000..b3ed537
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CachePartialUpdateException.cs
@@ -0,0 +1,119 @@
+/*
+ * 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.Cache
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Linq;
+    using System.Runtime.Serialization;
+
+    /// <summary>
+    /// Exception thrown from non-transactional cache in case when update succeeded only partially.
+    /// </summary>
+    [Serializable]
+    public class CachePartialUpdateException : CacheException
+    {
+        /** Serializer key. */
+        private const string KeyFailedKeys = "FailedKeys";
+
+        /** Failed keys. */
+        private readonly IList<object> _failedKeys;
+
+        /** Failed keys exception. */
+        private readonly Exception _failedKeysException;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CachePartialUpdateException"/> class.
+        /// </summary>
+        public CachePartialUpdateException()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CachePartialUpdateException"/> class.
+        /// </summary>
+        /// <param name="message">The message that describes the error.</param>
+        public CachePartialUpdateException(string message) : base(message)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CachePartialUpdateException"/> class.
+        /// </summary>
+        /// <param name="info">Serialization information.</param>
+        /// <param name="ctx">Streaming context.</param>
+        protected CachePartialUpdateException(SerializationInfo info, StreamingContext ctx)
+            : base(info, ctx)
+        {
+            _failedKeys = (IList<object>) info.GetValue(KeyFailedKeys, typeof (IList<object>));
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="msg">Exception message.</param>
+        /// <param name="failedKeysException">Exception occurred during failed keys read/write.</param>
+        public CachePartialUpdateException(string msg, Exception failedKeysException) : this(msg, null, failedKeysException)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="msg">Exception message.</param>
+        /// <param name="failedKeys">Failed keys.</param>
+        public CachePartialUpdateException(string msg, IList<object> failedKeys) : this(msg, failedKeys, null)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="msg">Exception message.</param>
+        /// <param name="failedKeys">Failed keys.</param>
+        /// <param name="failedKeysException">Exception occurred during failed keys read/write.</param>
+        private CachePartialUpdateException(string msg, IList<object> failedKeys, Exception failedKeysException) : base(msg)
+        {
+            _failedKeys = failedKeys;
+            _failedKeysException = failedKeysException;
+        }
+
+        /// <summary>
+        /// Gets the failed keys.
+        /// </summary>
+        public IEnumerable<T> GetFailedKeys<T>()
+        {
+            if (_failedKeysException != null)
+                throw _failedKeysException;
+            
+            return _failedKeys == null ? null : _failedKeys.Cast<T>();
+        }
+
+        /** <inheritdoc /> */
+        public override void GetObjectData(SerializationInfo info, StreamingContext context)
+        {
+            info.AddValue(KeyFailedKeys, _failedKeys);
+
+            base.GetObjectData(info, context);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CachePeekMode.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CachePeekMode.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CachePeekMode.cs
new file mode 100644
index 0000000..0a089ad
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CachePeekMode.cs
@@ -0,0 +1,68 @@
+/*
+ * 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.Cache
+{
+    using System;
+    using System.Diagnostics.CodeAnalysis;
+
+    /// <summary>
+    /// Enumeration of all supported cache peek modes.
+    /// </summary>
+    [Flags]
+    [SuppressMessage("Microsoft.Naming", "CA1714:FlagsEnumsShouldHavePluralNames")]
+    public enum CachePeekMode
+    {
+        /// <summary>
+        /// Peeks into all available cache storages.
+        /// </summary>
+        All = 0x01,
+
+        /// <summary>
+        /// Peek into near cache only (don't peek into partitioned cache).
+        /// In case of LOCAL cache, behaves as <see cref="All"/> mode.
+        /// </summary>
+        Near = 0x02,
+
+        /// <summary>
+        /// Peek value from primary copy of partitioned cache only (skip near cache).
+        /// In case of LOCAL cache, behaves as <see cref="All"/> mode.
+        /// </summary>
+        Primary = 0x04,
+
+        /// <summary>
+        /// Peek value from backup copies of partitioned cache only (skip near cache).
+        /// In case of LOCAL cache, behaves as <see cref="All"/> mode.
+        /// </summary>
+        Backup = 0x08,
+
+        /// <summary>
+        /// Peeks value from the on-heap storage only.
+        /// </summary>
+        Onheap = 0x10,
+
+        /// <summary>
+        /// Peeks value from the off-heap storage only, without loading off-heap value into cache.
+        /// </summary>
+        Offheap = 0x20,
+
+        /// <summary>
+        /// Peeks value from the swap storage only, without loading swapped value into cache.
+        /// </summary>
+        Swap = 0x40
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/CacheEntryEventType.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/CacheEntryEventType.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/CacheEntryEventType.cs
new file mode 100644
index 0000000..8339257
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/CacheEntryEventType.cs
@@ -0,0 +1,41 @@
+/*
+ * 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.Cache.Event
+{
+    /// <summary>
+    /// Cache event type.
+    /// </summary>
+    public enum CacheEntryEventType
+    {
+        /// <summary>
+        /// An event type indicating that the cache entry was created.
+        /// </summary>
+        Created,
+
+        /// <summary>
+        /// An event type indicating that the cache entry was updated. i.e. a previous
+        /// mapping existed.
+        /// </summary>
+        Updated,
+
+        /// <summary>
+        /// An event type indicating that the cache entry was removed.
+        /// </summary>
+        Removed
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/ICacheEntryEvent.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/ICacheEntryEvent.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/ICacheEntryEvent.cs
new file mode 100644
index 0000000..9c2665e
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/ICacheEntryEvent.cs
@@ -0,0 +1,40 @@
+/*
+ * 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.Cache.Event
+{
+    /// <summary>
+    /// Cache entry event.
+    /// </summary>
+    public interface ICacheEntryEvent<TK, TV> : ICacheEntry<TK, TV>
+    {
+        /// <summary>
+        /// Event type.
+        /// </summary>
+        CacheEntryEventType EventType { get; }
+
+        /// <summary>
+        /// Gets old the value.
+        /// </summary>
+        TV OldValue { get; }
+
+        /// <summary>
+        /// Whether old value exists.
+        /// </summary>
+        bool HasOldValue { get; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/ICacheEntryEventFilter.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/ICacheEntryEventFilter.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/ICacheEntryEventFilter.cs
new file mode 100644
index 0000000..98f5c5a
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/ICacheEntryEventFilter.cs
@@ -0,0 +1,31 @@
+/*
+ * 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.Cache.Event
+{
+    /// <summary>
+    /// Cache entry event filter.
+    /// </summary>
+    public interface ICacheEntryEventFilter<TK, TV>
+    {
+        /// <summary>
+        /// Evaluates cache entry event.
+        /// </summary>
+        /// <param name="evt">Event.</param>
+        bool Evaluate(ICacheEntryEvent<TK, TV> evt);
+    }
+}


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

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


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

Branch: refs/heads/ignite-843
Commit: b3e1b7da1b60a26dc5140efba6b4cef8da8ad56f
Parents: 39224cb 5cec202
Author: AKuznetsov <ak...@gridgain.com>
Authored: Fri Sep 4 21:15:57 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Fri Sep 4 21:15:57 2015 +0700

----------------------------------------------------------------------
 .../portable/PortableClassDescriptor.java       |   79 +-
 .../internal/portable/PortableContext.java      |  120 +-
 .../internal/portable/PortableWriterExImpl.java |   92 +-
 .../dotnet/PlatformDotNetConfiguration.java     |  119 +
 .../PlatformDotNetPortableConfiguration.java    |  228 ++
 ...PlatformDotNetPortableTypeConfiguration.java |  248 ++
 .../GridPortableMarshallerSelfTest.java         |   68 +-
 .../portable/GridPortableWildcardsSelfTest.java |   60 +-
 modules/platform/pom.xml                        |   14 +
 .../Apache.Ignite.Core.csproj                   |  283 +-
 .../Cache/CacheAtomicUpdateTimeoutException.cs  |   67 +
 .../Cache/CacheEntryProcessorException.cs       |   79 +
 .../Apache.Ignite.Core/Cache/CacheException.cs  |   68 +
 .../Cache/CachePartialUpdateException.cs        |  119 +
 .../Apache.Ignite.Core/Cache/CachePeekMode.cs   |   68 +
 .../Cache/Event/CacheEntryEventType.cs          |   41 +
 .../Cache/Event/ICacheEntryEvent.cs             |   40 +
 .../Cache/Event/ICacheEntryEventFilter.cs       |   31 +
 .../Cache/Event/ICacheEntryEventListener.cs     |   33 +
 .../Cache/Expiry/ExpiryPolicy.cs                |   89 +
 .../Cache/Expiry/IExpiryPolicy.cs               |   59 +
 .../dotnet/Apache.Ignite.Core/Cache/ICache.cs   |  542 +++
 .../Apache.Ignite.Core/Cache/ICacheAffinity.cs  |  161 +
 .../Apache.Ignite.Core/Cache/ICacheEntry.cs     |   37 +
 .../Cache/ICacheEntryFilter.cs                  |   34 +
 .../Cache/ICacheEntryProcessor.cs               |   45 +
 .../Cache/ICacheEntryProcessorResult.cs         |   40 +
 .../Apache.Ignite.Core/Cache/ICacheLock.cs      |   58 +
 .../Apache.Ignite.Core/Cache/ICacheMetrics.cs   |  486 +++
 .../Cache/IMutableCacheEntry.cs                 |   47 +
 .../Cache/Query/Continuous/ContinuousQuery.cs   |  170 +
 .../Query/Continuous/IContinuousQueryHandle.cs  |   51 +
 .../Cache/Query/IQueryCursor.cs                 |   40 +
 .../Apache.Ignite.Core/Cache/Query/QueryBase.cs |   82 +
 .../Apache.Ignite.Core/Cache/Query/ScanQuery.cs |   77 +
 .../Cache/Query/SqlFieldsQuery.cs               |   81 +
 .../Apache.Ignite.Core/Cache/Query/SqlQuery.cs  |  119 +
 .../Apache.Ignite.Core/Cache/Query/TextQuery.cs |  104 +
 .../Store/CacheParallelLoadStoreAdapter.cs      |  205 ++
 .../Cache/Store/CacheStoreAdapter.cs            |  146 +
 .../Cache/Store/CacheStoreException.cs          |   66 +
 .../Cache/Store/ICacheStore.cs                  |  184 +
 .../Cache/Store/ICacheStoreSession.cs           |   42 +
 .../Cluster/ClusterGroupEmptyException.cs       |   70 +
 .../Cluster/ClusterTopologyException.cs         |   69 +
 .../Apache.Ignite.Core/Cluster/ICluster.cs      |   80 +
 .../Apache.Ignite.Core/Cluster/IClusterGroup.cs |  229 ++
 .../Cluster/IClusterMetrics.cs                  |  515 +++
 .../Apache.Ignite.Core/Cluster/IClusterNode.cs  |  138 +
 .../Cluster/IClusterNodeFilter.cs               |   32 +
 .../Apache.Ignite.Core/Common/IAsyncSupport.cs  |    2 +-
 .../Common/IgniteException.cs                   |    2 +-
 .../Apache.Ignite.Core/Common/IgniteGuid.cs     |  138 +
 .../ComputeExecutionRejectedException.cs        |   69 +
 .../Compute/ComputeJobAdapter.cs                |  122 +
 .../Compute/ComputeJobFailoverException.cs      |   72 +
 .../Compute/ComputeJobResultPolicy.cs           |   45 +
 .../Compute/ComputeTaskAdapter.cs               |   93 +
 .../Compute/ComputeTaskCancelledException.cs    |   69 +
 .../ComputeTaskNoResultCacheAttribute.cs        |   35 +
 .../Compute/ComputeTaskSplitAdapter.cs          |   95 +
 .../Compute/ComputeTaskTimeoutException.cs      |   67 +
 .../Compute/ComputeUserUndeclaredException.cs   |   70 +
 .../Apache.Ignite.Core/Compute/ICompute.cs      |  274 ++
 .../Apache.Ignite.Core/Compute/IComputeFunc.cs  |   55 +
 .../Apache.Ignite.Core/Compute/IComputeJob.cs   |   58 +
 .../Compute/IComputeJobResult.cs                |   73 +
 .../Compute/IComputeReducer.cs                  |   39 +
 .../Apache.Ignite.Core/Compute/IComputeTask.cs  |  132 +
 .../Datastream/IDataStreamer.cs                 |  206 ++
 .../Datastream/IStreamReceiver.cs               |   38 +
 .../Datastream/StreamTransformer.cs             |   73 +
 .../Datastream/StreamVisitor.cs                 |   55 +
 .../Apache.Ignite.Core/Events/CacheEvent.cs     |  176 +
 .../Events/CacheQueryExecutedEvent.cs           |   97 +
 .../Events/CacheQueryReadEvent.cs               |  134 +
 .../Events/CacheRebalancingEvent.cs             |   98 +
 .../Events/CheckpointEvent.cs                   |   50 +
 .../Apache.Ignite.Core/Events/DiscoveryEvent.cs |   80 +
 .../Apache.Ignite.Core/Events/EventBase.cs      |  160 +
 .../Apache.Ignite.Core/Events/EventReader.cs    |   72 +
 .../Apache.Ignite.Core/Events/EventType.cs      |  514 +++
 .../dotnet/Apache.Ignite.Core/Events/IEvent.cs  |   74 +
 .../Apache.Ignite.Core/Events/IEventFilter.cs   |   36 +
 .../dotnet/Apache.Ignite.Core/Events/IEvents.cs |  182 +
 .../Apache.Ignite.Core/Events/JobEvent.cs       |  100 +
 .../Apache.Ignite.Core/Events/SwapSpaceEvent.cs |   50 +
 .../Apache.Ignite.Core/Events/TaskEvent.cs      |   91 +
 .../main/dotnet/Apache.Ignite.Core/IIgnite.cs   |  168 +
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  140 +
 .../main/dotnet/Apache.Ignite.Core/Ignition.cs  |  626 +++-
 .../Impl/Cache/CacheAffinityImpl.cs             |  275 ++
 .../Apache.Ignite.Core/Impl/Cache/CacheEntry.cs |  126 +
 .../Impl/Cache/CacheEntryFilterHolder.cs        |  147 +
 .../Impl/Cache/CacheEntryProcessorHolder.cs     |  145 +
 .../Impl/Cache/CacheEntryProcessorResult.cs     |   65 +
 .../Cache/CacheEntryProcessorResultHolder.cs    |  127 +
 .../Impl/Cache/CacheEnumerable.cs               |   82 +
 .../Impl/Cache/CacheEnumerator.cs               |  117 +
 .../Impl/Cache/CacheEnumeratorProxy.cs          |  156 +
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |  932 +++++
 .../Apache.Ignite.Core/Impl/Cache/CacheLock.cs  |  171 +
 .../Impl/Cache/CacheMetricsImpl.cs              |  248 ++
 .../Apache.Ignite.Core/Impl/Cache/CacheOp.cs    |   63 +
 .../Impl/Cache/CacheProxyImpl.cs                |  499 +++
 .../Impl/Cache/Event/CacheEntryCreateEvent.cs   |   74 +
 .../Impl/Cache/Event/CacheEntryRemoveEvent.cs   |   74 +
 .../Impl/Cache/Event/CacheEntryUpdateEvent.cs   |   79 +
 .../Impl/Cache/MutableCacheEntry.cs             |  163 +
 .../Impl/Cache/Query/AbstractQueryCursor.cs     |  264 ++
 .../Query/Continuous/ContinuousQueryFilter.cs   |  125 +
 .../Continuous/ContinuousQueryFilterHolder.cs   |  118 +
 .../Continuous/ContinuousQueryHandleImpl.cs     |  216 ++
 .../Query/Continuous/ContinuousQueryUtils.cs    |  115 +
 .../Impl/Cache/Query/FieldsQueryCursor.cs       |   54 +
 .../Impl/Cache/Query/QueryCursor.cs             |   50 +
 .../Impl/Cache/Store/CacheStore.cs              |  263 ++
 .../Impl/Cache/Store/CacheStoreSession.cs       |   53 +
 .../Impl/Cache/Store/CacheStoreSessionProxy.cs  |   63 +
 .../Impl/Cluster/ClusterGroupImpl.cs            |  577 +++
 .../Impl/Cluster/ClusterMetricsImpl.cs          |  292 ++
 .../Impl/Cluster/ClusterNodeImpl.cs             |  221 ++
 .../Impl/Cluster/IClusterGroupEx.cs             |   35 +
 .../Impl/Common/DelegateTypeDescriptor.cs       |  327 ++
 .../Apache.Ignite.Core/Impl/Common/Future.cs    |    3 +-
 .../Impl/Common/FutureConverter.cs              |   62 +
 .../Impl/Common/GridArgumentCheck.cs            |   76 -
 .../Impl/Common/IgniteArgumentCheck.cs          |   76 +
 .../Impl/Common/PortableResultWrapper.cs        |   68 +
 .../Closure/ComputeAbstractClosureTask.cs       |  101 +
 .../Impl/Compute/Closure/ComputeActionJob.cs    |   83 +
 .../Impl/Compute/Closure/ComputeFuncJob.cs      |   89 +
 .../Compute/Closure/ComputeMultiClosureTask.cs  |   56 +
 .../Impl/Compute/Closure/ComputeOutFuncJob.cs   |   76 +
 .../Closure/ComputeReducingClosureTask.cs       |   61 +
 .../Compute/Closure/ComputeSingleClosureTask.cs |   48 +
 .../Compute/Closure/IComputeResourceInjector.cs |   31 +
 .../Apache.Ignite.Core/Impl/Compute/Compute.cs  |  213 ++
 .../Impl/Compute/ComputeAsync.cs                |  261 ++
 .../Impl/Compute/ComputeFunc.cs                 |  119 +
 .../Impl/Compute/ComputeImpl.cs                 |  645 ++++
 .../Impl/Compute/ComputeJob.cs                  |  163 +
 .../Impl/Compute/ComputeJobHolder.cs            |  240 ++
 .../Compute/ComputeJobResultGenericWrapper.cs   |   70 +
 .../Impl/Compute/ComputeJobResultImpl.cs        |   96 +
 .../Impl/Compute/ComputeOutFunc.cs              |  123 +
 .../Impl/Compute/ComputeTaskHolder.cs           |  484 +++
 .../Impl/Datastream/DataStreamerBatch.cs        |  269 ++
 .../Impl/Datastream/DataStreamerEntry.cs        |   64 +
 .../Impl/Datastream/DataStreamerImpl.cs         |  832 +++++
 .../Impl/Datastream/DataStreamerRemoveEntry.cs  |   48 +
 .../Impl/Datastream/StreamReceiverHolder.cs     |  144 +
 .../Apache.Ignite.Core/Impl/Events/Events.cs    |  498 +++
 .../Impl/Events/EventsAsync.cs                  |  158 +
 .../Impl/Events/RemoteListenEventFilter.cs      |   85 +
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs   |  204 ++
 .../Impl/Handle/HandleRegistry.cs               |    4 +-
 .../Apache.Ignite.Core/Impl/IInteropCallback.cs |   34 +
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |  547 +++
 .../Impl/IgniteConfigurationEx.cs               |   57 +
 .../Apache.Ignite.Core/Impl/IgniteManager.cs    |  492 +++
 .../Apache.Ignite.Core/Impl/IgniteProxy.cs      |  351 ++
 .../Apache.Ignite.Core/Impl/IgniteUtils.cs      |  438 +++
 .../Impl/Interop/InteropDotNetConfiguration.cs  |   62 +
 .../InteropDotNetPortableConfiguration.cs       |  127 +
 .../InteropDotNetPortableTypeConfiguration.cs   |  151 +
 .../Impl/InteropExceptionHolder.cs              |   85 +
 .../Impl/LifecycleBeanHolder.cs                 |   66 +
 .../Impl/Memory/InteropExternalMemory.cs        |   46 +
 .../Impl/Memory/InteropMemoryUtils.cs           |   38 +
 .../Impl/Memory/PlatformMemoryManager.cs        |    5 +-
 .../Impl/Memory/PlatformMemoryStream.cs         |    3 +-
 .../Impl/Memory/PlatformMemoryUtils.cs          |    2 +-
 .../Impl/Memory/PlatformPooledMemory.cs         |    2 +-
 .../Impl/Messaging/MessageFilterHolder.cs       |  179 +
 .../Impl/Messaging/Messaging.cs                 |  262 ++
 .../Impl/Messaging/MessagingAsync.cs            |   68 +
 .../Apache.Ignite.Core/Impl/NativeMethods.cs    |   47 +
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |  715 ++++
 .../Portable/IPortableSystemTypeSerializer.cs   |   34 +
 .../Impl/Portable/IPortableTypeDescriptor.cs    |  124 +
 .../Impl/Portable/IPortableWriteAware.cs        |   34 +
 .../Impl/Portable/Io/PortableAbstractStream.cs  | 1299 +++++++
 .../Impl/Portable/Io/PortableHeapStream.cs      |  447 +++
 .../Impl/Portable/Io/PortableStreamAdapter.cs   |  114 +
 .../Metadata/IPortableMetadataHandler.cs        |   41 +
 .../Metadata/PortableHashsetMetadataHandler.cs  |   69 +
 .../Portable/Metadata/PortableMetadataHolder.cs |  149 +
 .../Portable/Metadata/PortableMetadataImpl.cs   |  200 ++
 .../Impl/Portable/PortableBuilderField.cs       |   73 +
 .../Impl/Portable/PortableBuilderImpl.cs        |  923 +++++
 .../Impl/Portable/PortableCollectionInfo.cs     |  251 ++
 .../Impl/Portable/PortableFullTypeDescriptor.cs |  203 ++
 .../Impl/Portable/PortableHandleDictionary.cs   |  187 +
 .../Portable/PortableMarshalAwareSerializer.cs  |   45 +
 .../Impl/Portable/PortableMarshaller.cs         |  603 ++++
 .../Impl/Portable/PortableMode.cs               |   40 +
 .../Impl/Portable/PortableObjectHandle.cs       |   59 +
 .../PortableOrSerializableObjectHolder.cs       |   66 +
 .../Portable/PortableReaderHandleDictionary.cs  |   42 +
 .../Impl/Portable/PortableReaderImpl.cs         | 1013 ++++++
 .../Impl/Portable/PortableReflectiveRoutines.cs |  483 +++
 .../Portable/PortableReflectiveSerializer.cs    |  218 ++
 .../Portable/PortableSurrogateTypeDescriptor.cs |  133 +
 .../Impl/Portable/PortableSystemHandlers.cs     | 1336 +++++++
 .../Portable/PortableSystemTypeSerializer.cs    |   62 +
 .../Impl/Portable/PortableUserObject.cs         |  385 ++
 .../Impl/Portable/PortableUtils.cs              | 2039 +++++++++++
 .../Impl/Portable/PortableWriterImpl.cs         | 1305 +++++++
 .../Impl/Portable/PortablesImpl.cs              |  205 ++
 .../Impl/Portable/SerializableObjectHolder.cs   |   66 +
 .../Impl/Portable/TypeResolver.cs               |  227 ++
 .../Impl/Resource/IResourceInjector.cs          |   27 +
 .../Impl/Resource/ResourceFieldInjector.cs      |   47 +
 .../Impl/Resource/ResourceMethodInjector.cs     |   48 +
 .../Impl/Resource/ResourceProcessor.cs          |  105 +
 .../Impl/Resource/ResourcePropertyInjector.cs   |   47 +
 .../Impl/Resource/ResourceTypeDescriptor.cs     |  291 ++
 .../Impl/Services/ServiceContext.cs             |   60 +
 .../Impl/Services/ServiceDescriptor.cs          |  106 +
 .../Impl/Services/ServiceProxy.cs               |   71 +
 .../Impl/Services/ServiceProxyInvoker.cs        |  136 +
 .../Impl/Services/ServiceProxySerializer.cs     |  140 +
 .../Impl/Services/Services.cs                   |  316 ++
 .../Impl/Services/ServicesAsync.cs              |   89 +
 .../Impl/Transactions/AsyncTransaction.cs       |   78 +
 .../Impl/Transactions/Transaction.cs            |  155 +
 .../Impl/Transactions/TransactionImpl.cs        |  489 +++
 .../Impl/Transactions/TransactionMetricsImpl.cs |   62 +
 .../Impl/Transactions/TransactionsImpl.cs       |  201 ++
 .../Impl/Unmanaged/IUnmanagedTarget.cs          |   42 +
 .../Impl/Unmanaged/UnmanagedCallbackHandlers.cs |   99 +
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        | 1152 ++++++
 .../Impl/Unmanaged/UnmanagedContext.cs          |   53 +
 .../Unmanaged/UnmanagedNonReleaseableTarget.cs  |   68 +
 .../Impl/Unmanaged/UnmanagedTarget.cs           |   77 +
 .../Impl/Unmanaged/UnmanagedUtils.cs            | 1252 +++++++
 .../Lifecycle/ILifecycleBean.cs                 |   64 +
 .../Lifecycle/LifecycleEventType.cs             |   49 +
 .../Messaging/IMessageFilter.cs                 |   35 +
 .../Apache.Ignite.Core/Messaging/IMessaging.cs  |  105 +
 .../Portable/IPortableBuilder.cs                |   78 +
 .../Portable/IPortableIdMapper.cs               |   40 +
 .../Portable/IPortableMarshalAware.cs           |   39 +
 .../Portable/IPortableMetadata.cs               |   61 +
 .../Portable/IPortableNameMapper.cs             |   39 +
 .../Portable/IPortableObject.cs                 |   44 +
 .../Portable/IPortableRawReader.cs              |  264 ++
 .../Portable/IPortableRawWriter.cs              |  221 ++
 .../Portable/IPortableReader.cs                 |  340 ++
 .../Portable/IPortableSerializer.cs             |   39 +
 .../Portable/IPortableWriter.cs                 |  259 ++
 .../Apache.Ignite.Core/Portable/IPortables.cs   |  120 +
 .../Portable/PortableConfiguration.cs           |  122 +
 .../Portable/PortableException.cs               |   64 +
 .../Portable/PortableTypeConfiguration.cs       |  162 +
 .../Portable/PortableTypeNames.cs               |  115 +
 .../Resource/InstanceResourceAttribute.cs       |   35 +
 .../Resource/StoreSessionResourceAttribute.cs   |   32 +
 .../Apache.Ignite.Core/Services/IService.cs     |   51 +
 .../Services/IServiceContext.cs                 |   69 +
 .../Services/IServiceDescriptor.cs              |   96 +
 .../Apache.Ignite.Core/Services/IServices.cs    |  181 +
 .../Services/ServiceConfiguration.cs            |   62 +
 .../Services/ServiceInvocationException.cs      |  101 +
 .../Transactions/ITransaction.cs                |  230 ++
 .../Transactions/ITransactionMetrics.cs         |   47 +
 .../Transactions/ITransactions.cs               |   73 +
 .../Transactions/TransactionConcurrency.cs      |   36 +
 .../TransactionHeuristicException.cs            |   72 +
 .../Transactions/TransactionIsolation.cs        |   41 +
 .../TransactionOptimisticException.cs           |   69 +
 .../TransactionRollbackException.cs             |   68 +
 .../Transactions/TransactionState.cs            |   70 +
 .../Transactions/TransactionTimeoutException.cs |   69 +
 .../platform/src/main/dotnet/Apache.Ignite.sln  |   27 +-
 .../dotnet/PlatformDotNetConfiguration.java     |  119 -
 .../PlatformDotNetPortableConfiguration.java    |  228 --
 ...PlatformDotNetPortableTypeConfiguration.java |  248 --
 .../Apache.Ignite.Core.Tests.TestDll.csproj     |   52 +
 .../Properties/AssemblyInfo.cs                  |   49 +
 .../TestClass.cs                                |   35 +
 .../Apache.Ignite.Core.Tests.csproj             |  165 +-
 .../Cache/CacheAbstractTest.cs                  | 3321 ++++++++++++++++++
 .../Cache/CacheAffinityTest.cs                  |  139 +
 .../Cache/CacheDynamicStartTest.cs              |  281 ++
 .../Cache/CacheEntryTest.cs                     |   69 +
 .../Cache/CacheForkedTest.cs                    |   82 +
 .../Cache/CacheLocalAtomicTest.cs               |   57 +
 .../Cache/CacheLocalTest.cs                     |   56 +
 .../CachePartitionedAtomicNearEnabledTest.cs    |   50 +
 .../Cache/CachePartitionedAtomicTest.cs         |   50 +
 .../Cache/CachePartitionedNearEnabledTest.cs    |   50 +
 .../Cache/CachePartitionedTest.cs               |   50 +
 .../Cache/CacheReplicatedAtomicTest.cs          |   60 +
 .../Cache/CacheReplicatedTest.cs                |   60 +
 .../Cache/CacheTestAsyncWrapper.cs              |  436 +++
 .../Cache/Query/CacheQueriesTest.cs             |  928 +++++
 .../Continuous/ContinuousQueryAbstractTest.cs   | 1181 +++++++
 .../ContinuousQueryAtomicBackupTest.cs          |   33 +
 .../ContinuousQueryAtomicNoBackupTest.cs        |   34 +
 .../ContinuousQueryNoBackupAbstractTest.cs      |   72 +
 .../ContinuousQueryTransactionalBackupTest.cs   |   34 +
 .../ContinuousQueryTransactionalNoBackupTest.cs |   33 +
 .../Cache/Store/CacheParallelLoadStoreTest.cs   |  110 +
 .../Cache/Store/CacheStoreSessionTest.cs        |  285 ++
 .../Cache/Store/CacheStoreTest.cs               |  510 +++
 .../Cache/Store/CacheTestParallelLoadStore.cs   |   91 +
 .../Cache/Store/CacheTestStore.cs               |  155 +
 .../Compute/AbstractTaskTest.cs                 |  217 ++
 .../Compute/ClosureTaskTest.cs                  |  390 ++
 .../Compute/ComputeApiTest.cs                   | 1281 +++++++
 .../Compute/ComputeMultithreadedTest.cs         |  269 ++
 .../Compute/FailoverTaskSelfTest.cs             |  246 ++
 .../Forked/ForkedPortableClosureTaskTest.cs     |   33 +
 .../Compute/Forked/ForkedResourceTaskTest.cs    |   33 +
 .../Forked/ForkedSerializableClosureTaskTest.cs |   33 +
 .../Compute/Forked/ForkedTaskAdapterTest.cs     |   33 +
 .../Compute/IgniteExceptionTaskSelfTest.cs      |  753 ++++
 .../Compute/PortableClosureTaskTest.cs          |  217 ++
 .../Compute/PortableTaskTest.cs                 |  253 ++
 .../Compute/ResourceTaskTest.cs                 |  568 +++
 .../Compute/SerializableClosureTaskTest.cs      |  217 ++
 .../Compute/TaskAdapterTest.cs                  |  274 ++
 .../Compute/TaskResultTest.cs                   |  437 +++
 .../Config/Cache/Store/cache-store-session.xml  |   80 +
 .../Config/Compute/compute-grid1.xml            |   90 +
 .../Config/Compute/compute-grid2.xml            |   63 +
 .../Config/Compute/compute-grid3.xml            |   52 +
 .../Config/Compute/compute-standalone.xml       |   87 +
 .../Config/Dynamic/dynamic-client.xml           |   51 +
 .../Config/Dynamic/dynamic-data-no-cfg.xml      |   47 +
 .../Config/Dynamic/dynamic-data.xml             |   65 +
 .../Config/Ignite.exe.config.test               |   41 +
 .../Config/Lifecycle/lifecycle-beans.xml        |   66 +
 .../Config/Lifecycle/lifecycle-no-beans.xml     |   44 +
 .../Config/cache-portables.xml                  |   78 +
 .../Config/cache-query-continuous.xml           |  171 +
 .../Config/cache-query.xml                      |  100 +
 .../Config/marshaller-default.xml               |   43 +
 .../Config/marshaller-invalid.xml               |   46 +
 .../Config/marshaller-portable.xml              |   43 +
 .../native-client-test-cache-affinity.xml       |   70 +
 .../native-client-test-cache-parallel-store.xml |   69 +
 .../native-client-test-cache-portables.xml      |  226 ++
 .../Config/native-client-test-cache-store.xml   |  125 +
 .../Config/native-client-test-cache.xml         |  224 ++
 .../Config/portable.xml                         |   56 +
 .../Config/start-test-grid1.xml                 |   54 +
 .../Config/start-test-grid2.xml                 |   45 +
 .../Config/start-test-grid3.xml                 |   43 +
 .../Dataload/DataStreamerTest.cs                |  592 ++++
 .../Apache.Ignite.Core.Tests/EventsTest.cs      |  961 +++++
 .../Apache.Ignite.Core.Tests/ExceptionsTest.cs  |  352 ++
 .../Apache.Ignite.Core.Tests/ExecutableTest.cs  |  444 +++
 .../Apache.Ignite.Core.Tests/FutureTest.cs      |  278 ++
 .../IgniteManagerTest.cs                        |   51 +
 .../IgniteStartStopTest.cs                      |  380 ++
 .../Apache.Ignite.Core.Tests/IgnitionTest.cs    |   30 -
 .../Apache.Ignite.Core.Tests/LifecycleTest.cs   |  288 ++
 .../Apache.Ignite.Core.Tests/LoadDllTest.cs     |  243 ++
 .../Apache.Ignite.Core.Tests/MarshallerTest.cs  |   71 +
 .../Apache.Ignite.Core.Tests/MessagingTest.cs   |  646 ++++
 .../Portable/PortableApiSelfTest.cs             | 1787 ++++++++++
 .../Portable/PortableSelfTest.cs                | 2078 +++++++++++
 .../PortableConfigurationTest.cs                |  173 +
 .../Process/IIgniteProcessOutputReader.cs       |   35 +
 .../Process/IgniteProcess.cs                    |  292 ++
 .../Process/IgniteProcessConsoleOutputReader.cs |   40 +
 .../Query/ImplicitPortablePerson.cs             |   46 +
 .../Query/NoDefPortablePerson.cs                |   35 +
 .../Query/PortablePerson.cs                     |   69 +
 .../SerializationTest.cs                        |  240 ++
 .../Services/ServiceProxyTest.cs                |  741 ++++
 .../Services/ServicesAsyncWrapper.cs            |  174 +
 .../Services/ServicesTest.cs                    |  823 +++++
 .../Services/ServicesTestAsync.cs               |   33 +
 .../Apache.Ignite.Core.Tests/TestRunner.cs      |   15 +-
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |  292 ++
 .../TypeResolverTest.cs                         |  107 +
 .../platform/PlatformComputeBroadcastTask.java  |   73 +
 .../platform/PlatformComputeDecimalTask.java    |  106 +
 .../platform/PlatformComputeEchoTask.java       |  188 +
 .../ignite/platform/PlatformComputeEnum.java    |   28 +
 .../platform/PlatformComputeJavaPortable.java   |   39 +
 .../platform/PlatformComputePortable.java       |   42 +
 .../PlatformComputePortableArgTask.java         |  119 +
 .../platform/PlatformEventsWriteEventTask.java  |  146 +
 .../ignite/platform/PlatformMaxMemoryTask.java  |   57 +
 .../ignite/platform/PlatformMinMemoryTask.java  |   57 +
 .../lifecycle/PlatformJavaLifecycleBean.java    |   47 +
 .../lifecycle/PlatformJavaLifecycleTask.java    |   65 +
 parent/pom.xml                                  |    6 +
 393 files changed, 75515 insertions(+), 933 deletions(-)
----------------------------------------------------------------------


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


[36/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/QueryBase.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/QueryBase.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/QueryBase.cs
new file mode 100644
index 0000000..3cb9e58
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/QueryBase.cs
@@ -0,0 +1,82 @@
+/*
+ * 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.Cache.Query
+{
+    using Apache.Ignite.Core.Impl.Cache;
+    using Apache.Ignite.Core.Impl.Portable;
+
+    /// <summary>
+    /// Base class for all Ignite cache entry queries.
+    /// </summary>
+    public abstract class QueryBase
+    {
+        /** Default page size. */
+        public const int DfltPageSize = 1024;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="QueryBase"/> class.
+        /// </summary>
+        protected internal QueryBase()
+        {
+            PageSize = DfltPageSize;
+        }
+
+        /// <summary>
+        /// Local flag. When set query will be executed only on local node, so only local 
+        /// entries will be returned as query result.
+        /// <para />
+        /// Defaults to <c>false</c>.
+        /// </summary>
+        public bool Local { get; set; }
+
+        /// <summary>
+        /// Optional page size. If set to <code>0</code>, then <code>CacheQueryConfiguration.pageSize</code> is used.
+        /// </summary>
+        public int PageSize { get; set; }
+
+        /// <summary>
+        /// Writes this instance to a stream created with a specified delegate.
+        /// </summary>
+        /// <param name="writer">Writer.</param>
+        /// <param name="keepPortable">Keep portable flag.</param>
+        internal abstract void Write(PortableWriterImpl writer, bool keepPortable);
+
+        /// <summary>
+        /// Gets the interop opcode.
+        /// </summary>
+        internal abstract CacheOp OpId { get; }
+
+        /// <summary>
+        /// Write query arguments.
+        /// </summary>
+        /// <param name="writer">Writer.</param>
+        /// <param name="args">Arguments.</param>
+        internal static void WriteQueryArgs(PortableWriterImpl writer, object[] args)
+        {
+            if (args == null)
+                writer.WriteInt(0);
+            else
+            {
+                writer.WriteInt(args.Length);
+
+                foreach (var arg in args)
+                    writer.WriteObject(arg);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/ScanQuery.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/ScanQuery.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/ScanQuery.cs
new file mode 100644
index 0000000..44f8486
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/ScanQuery.cs
@@ -0,0 +1,77 @@
+/*
+ * 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.Cache.Query
+{
+    using Apache.Ignite.Core.Impl.Cache;
+    using Apache.Ignite.Core.Impl.Portable;
+
+    /// <summary>
+    /// Scan query over cache entries. Will accept all the entries if no predicate was set.
+    /// </summary>
+    public class ScanQuery<TK, TV> : QueryBase
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ScanQuery{K, V}"/> class.
+        /// </summary>
+        /// <param name="filter">The filter.</param>
+        public ScanQuery(ICacheEntryFilter<TK, TV> filter = null)
+        {
+            Filter = filter;
+        }
+
+        /// <summary>
+        /// Gets or sets the predicate.
+        /// </summary>
+        public ICacheEntryFilter<TK, TV> Filter { get; set; }
+
+        /// <summary>
+        /// Gets or sets partition number over which this query should iterate. If null, query will iterate 
+        /// over all partitions in the cache. Must be in the range [0, N) where N is partition number in the cache.
+        /// </summary>
+        public int? Partition { get; set; }
+
+        /** <inheritDoc /> */
+        internal override void Write(PortableWriterImpl writer, bool keepPortable)
+        {
+            writer.WriteBoolean(Local);
+            writer.WriteInt(PageSize);
+            
+            writer.WriteBoolean(Partition.HasValue);
+            
+            if (Partition.HasValue)
+                writer.WriteInt(Partition.Value);
+
+            if (Filter == null)
+                writer.WriteObject<CacheEntryFilterHolder>(null);
+            else
+            {
+                var holder = new CacheEntryFilterHolder(Filter, (key, val) => Filter.Invoke(
+                    new CacheEntry<TK, TV>((TK) key, (TV) val)), writer.Marshaller, keepPortable);
+                
+                writer.WriteObject(holder);
+                writer.WriteLong(holder.Handle);
+            }
+        }
+
+        /** <inheritDoc /> */
+        internal override CacheOp OpId
+        {
+            get { return CacheOp.QryScan; }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/SqlFieldsQuery.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/SqlFieldsQuery.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/SqlFieldsQuery.cs
new file mode 100644
index 0000000..c0d58ca
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/SqlFieldsQuery.cs
@@ -0,0 +1,81 @@
+/*
+ * 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.Cache.Query
+{
+    using System.Diagnostics.CodeAnalysis;
+
+    /// <summary>
+    /// SQL fields query.
+    /// </summary>
+    public class SqlFieldsQuery
+    {
+        /** Default page size. */
+        public const int DfltPageSize = 1024;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="sql">SQL.</param>
+        /// <param name="args">Arguments.</param>
+        public SqlFieldsQuery(string sql, params object[] args) : this(sql, false, args)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Constructor,
+        /// </summary>
+        /// <param name="sql">SQL.</param>
+        /// <param name="loc">Whether query should be executed locally.</param>
+        /// <param name="args">Arguments.</param>
+        public SqlFieldsQuery(string sql, bool loc, params object[] args)
+        {
+            Sql = sql;
+            Local = loc;
+            Arguments = args;
+
+            PageSize = DfltPageSize;
+        }
+
+        /// <summary>
+        /// SQL.
+        /// </summary>
+        public string Sql { get; set; }
+        
+        /// <summary>
+        /// Arguments.
+        /// </summary>
+        [SuppressMessage("Microsoft.Performance", "CA1819:PropertiesShouldNotReturnArrays")]
+        public object[] Arguments { get; set; }
+
+        /// <summary>
+        /// Local flag. When set query will be executed only on local node, so only local 
+        /// entries will be returned as query result.
+        /// <para />
+        /// Defaults to <c>false</c>.
+        /// </summary>
+        public bool Local { get; set; }
+
+        /// <summary>
+        /// Optional page size.
+        /// <para />
+        /// Defautls to <see cref="DfltPageSize"/>.
+        /// </summary>
+        public int PageSize { get; set; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/SqlQuery.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/SqlQuery.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/SqlQuery.cs
new file mode 100644
index 0000000..303048b
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/SqlQuery.cs
@@ -0,0 +1,119 @@
+/*
+ * 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.Cache.Query
+{
+    using System;
+    using System.Diagnostics.CodeAnalysis;
+    using Apache.Ignite.Core.Impl.Cache;
+    using Apache.Ignite.Core.Impl.Portable;
+
+    /// <summary>
+    /// SQL Query.
+    /// </summary>
+    public class SqlQuery : QueryBase
+    {
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="typ">Type.</param>
+        /// <param name="sql">SQL.</param>
+        /// <param name="args">Arguments.</param>
+        public SqlQuery(Type typ, string sql, params object[] args) : this(typ, sql, false, args)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="typ">Type.</param>
+        /// <param name="sql">SQL.</param>
+        /// <param name="loc">Whether query should be executed locally.</param>
+        /// <param name="args">Arguments.</param>
+        public SqlQuery(Type typ, string sql, bool loc, params object[] args) : this(typ.Name, sql, loc, args)
+        {
+            // No-op.
+        }
+        
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="typ">Type.</param>
+        /// <param name="sql">SQL.</param>
+        /// <param name="args">Arguments.</param>
+        public SqlQuery(string typ, string sql, params object[] args) : this(typ, sql, false, args)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="typ">Type.</param>
+        /// <param name="sql">SQL.</param>
+        /// <param name="loc">Whether query should be executed locally.</param>
+        /// <param name="args">Arguments.</param>
+        public SqlQuery(string typ, string sql, bool loc, params object[] args)
+        {
+            Type = typ;
+            Sql = sql;
+            Local = loc;
+            Arguments = args;
+        }
+
+        /// <summary>
+        /// Type.
+        /// </summary>
+        public string Type { get; set; }
+
+        /// <summary>
+        /// SQL.
+        /// </summary>
+        public string Sql { get; set; }
+
+        /// <summary>
+        /// Arguments.
+        /// </summary>
+        [SuppressMessage("Microsoft.Performance", "CA1819:PropertiesShouldNotReturnArrays")]
+        public object[] Arguments { get; set; }
+
+        /** <inheritDoc /> */
+        internal override void Write(PortableWriterImpl writer, bool keepPortable)
+        {
+            if (string.IsNullOrEmpty(Sql))
+                throw new ArgumentException("Sql cannot be null or empty");
+
+            if (string.IsNullOrEmpty(Type))
+                throw new ArgumentException("Type cannot be null or empty");
+
+            // 2. Prepare.
+            writer.WriteBoolean(Local);
+            writer.WriteString(Sql);
+            writer.WriteString(Type);
+            writer.WriteInt(PageSize);
+
+            WriteQueryArgs(writer, Arguments);
+        }
+
+        /** <inheritDoc /> */
+        internal override CacheOp OpId
+        {
+            get { return CacheOp.QrySql; }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/TextQuery.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/TextQuery.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/TextQuery.cs
new file mode 100644
index 0000000..835271b
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Query/TextQuery.cs
@@ -0,0 +1,104 @@
+/*
+ * 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.Cache.Query
+{
+    using System;
+    using Apache.Ignite.Core.Impl.Cache;
+    using Apache.Ignite.Core.Impl.Portable;
+
+    /// <summary>
+    /// Text query.
+    /// </summary>
+    public class TextQuery : QueryBase
+    {
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="typ">Type.</param>
+        /// <param name="txt">Text.</param>
+        public TextQuery(Type typ, string txt) : this(typ, txt, false)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="typ">Type.</param>
+        /// <param name="txt">Text.</param>
+        /// <param name="loc">Whether query should be executed locally.</param>
+        public TextQuery(Type typ, string txt, bool loc) : this(typ.Name, txt, loc)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="typ">Type.</param>
+        /// <param name="txt">Text.</param>
+        public TextQuery(string typ, string txt) : this(typ, txt, false)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="typ">Type.</param>
+        /// <param name="txt">Text.</param>
+        /// <param name="loc">Whether query should be executed locally.</param>
+        public TextQuery(string typ, string txt, bool loc)
+        {
+            Type = typ;
+            Text = txt;
+            Local = loc;
+        }
+
+        /// <summary>
+        /// Type.
+        /// </summary>
+        public string Type { get; set; }
+
+        /// <summary>
+        /// Text.
+        /// </summary>
+        public string Text { get; set; }
+
+        /** <inheritDoc /> */
+        internal override void Write(PortableWriterImpl writer, bool keepPortable)
+        {
+            if (string.IsNullOrEmpty(Text))
+                throw new ArgumentException("Text cannot be null or empty");
+
+            if (string.IsNullOrEmpty(Type))
+                throw new ArgumentException("Type cannot be null or empty");
+
+            writer.WriteBoolean(Local);
+            writer.WriteString(Text);
+            writer.WriteString(Type);
+            writer.WriteInt(PageSize);
+        }
+
+        /** <inheritDoc /> */
+        internal override CacheOp OpId
+        {
+            get { return CacheOp.QryTxt; }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Store/CacheParallelLoadStoreAdapter.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Store/CacheParallelLoadStoreAdapter.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Store/CacheParallelLoadStoreAdapter.cs
new file mode 100644
index 0000000..cf4a77d
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Store/CacheParallelLoadStoreAdapter.cs
@@ -0,0 +1,205 @@
+/*
+ * 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.Cache.Store
+{
+    using System;
+    using System.Collections;
+    using System.Collections.Generic;
+    using System.Linq;
+    using System.Threading.Tasks;
+
+    /// <summary>
+    /// Cache storage adapter with parallel loading in LoadAll method. 
+    /// </summary>
+    /// <remarks>
+    /// LoadCache calls GetInputData() and iterates over it in parallel.
+    /// GetInputData().GetEnumerator() result will be disposed if it implements IDisposable.
+    /// Any additional post-LoadCache steps can be performed by overriding LoadCache method.
+    /// </remarks>
+    public abstract class CacheParallelLoadStoreAdapter : ICacheStore
+    {
+        /// <summary>
+        /// Default number of working threads (equal to the number of available processors).
+        /// </summary>
+        public static readonly int DefaultThreadsCount = Environment.ProcessorCount;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        protected CacheParallelLoadStoreAdapter()
+        {
+            MaxDegreeOfParallelism = DefaultThreadsCount;
+        }
+
+        /// <summary>
+        /// Loads all values from underlying persistent storage. Note that keys are
+        /// not passed, so it is up to implementation to figure out what to load.
+        /// This method is called whenever <see cref="ICache{K,V}.LocalLoadCache" />
+        /// method is invoked which is usually to preload the cache from persistent storage.
+        /// <para />
+        /// This method is optional, and cache implementation
+        /// does not depend on this method to do anything.
+        /// <para />
+        /// For every loaded value method provided action should be called.
+        /// The action will then make sure that the loaded value is stored in cache.
+        /// </summary>
+        /// <param name="act">Action for loaded values.</param>
+        /// <param name="args">Optional arguemnts passed to <see cref="ICache{K,V}.LocalLoadCache" /> method.</param>
+        /// <exception cref="CacheStoreException" />
+        public virtual void LoadCache(Action<object, object> act, params object[] args)
+        {
+            if (MaxDegreeOfParallelism == 0 || MaxDegreeOfParallelism < -1)
+                throw new ArgumentOutOfRangeException("MaxDegreeOfParallelism must be either positive or -1: " +
+                                                      MaxDegreeOfParallelism);
+
+            var options = new ParallelOptions {MaxDegreeOfParallelism = MaxDegreeOfParallelism};
+
+            Parallel.ForEach(GetInputData().OfType<object>(), options, item =>
+            {
+                var cacheEntry = Parse(item, args);
+
+                if (cacheEntry != null)
+                    act(cacheEntry.Value.Key, cacheEntry.Value.Value);
+            });
+        }
+
+        /// <summary>
+        /// Gets the input data sequence to be used in LoadCache.
+        /// </summary>
+        protected abstract IEnumerable GetInputData();
+
+        /// <summary>
+        /// This method should transform raw data records from GetInputData
+        /// into valid key-value pairs to be stored into cache.        
+        /// </summary>
+        protected abstract KeyValuePair<object, object>? Parse(object inputRecord, params object[] args);
+
+        /// <summary>
+        /// Gets or sets the maximum degree of parallelism to use in LoadCache. 
+        /// Must be either positive or -1 for unlimited amount of threads.
+        /// <para />
+        /// Defaults to <see cref="DefaultThreadsCount"/>.
+        /// </summary>
+        public int MaxDegreeOfParallelism { get; set; }
+
+        /// <summary>
+        /// Loads an object. Application developers should implement this method to customize the loading
+        /// of a value for a cache entry.
+        /// This method is called by a cache when a requested entry is not in the cache.
+        /// If the object can't be loaded <code>null</code> should be returned.
+        /// </summary>
+        /// <param name="key">The key identifying the object being loaded.</param>
+        /// <returns>
+        /// The value for the entry that is to be stored in the cache
+        /// or <code>null</code> if the object can't be loaded
+        /// </returns>
+        public virtual object Load(object key)
+        {
+            return null;
+        }
+
+        /// <summary>
+        /// Loads multiple objects. Application developers should implement this method to customize
+        /// the loading of cache entries. This method is called when the requested object is not in the cache.
+        /// If an object can't be loaded, it is not returned in the resulting map.
+        /// </summary>
+        /// <param name="keys">Keys identifying the values to be loaded.</param>
+        /// <returns>
+        /// A map of key, values to be stored in the cache.
+        /// </returns>
+        public virtual IDictionary LoadAll(ICollection keys)
+        {
+            return null;
+        }
+
+        /// <summary>
+        /// Write the specified value under the specified key to the external resource.
+        /// <para />
+        /// This method is intended to support both key/value creation and value update.
+        /// </summary>
+        /// <param name="key">Key to write.</param>
+        /// <param name="val">Value to write.</param>
+        public virtual void Write(object key, object val)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Write the specified entries to the external resource.
+        /// This method is intended to support both insert and update.
+        /// <para />
+        /// The order that individual writes occur is undefined.
+        /// <para />
+        /// If this operation fails (by throwing an exception) after a partial success,
+        /// the writer must remove any successfully written entries from the entries collection
+        /// so that the caching implementation knows what succeeded and can mutate the cache.
+        /// </summary>
+        /// <param name="entries">a mutable collection to write. Upon invocation,  it contains the entries
+        /// to write for write-through. Upon return the collection must only contain entries
+        /// that were not successfully written. (see partial success above).</param>
+        public virtual void WriteAll(IDictionary entries)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Delete the cache entry from the external resource.
+        /// <para />
+        /// Expiry of a cache entry is not a delete hence will not cause this method to be invoked.
+        /// <para />
+        /// This method is invoked even if no mapping for the key exists.
+        /// </summary>
+        /// <param name="key">The key that is used for the delete operation.</param>
+        public virtual void Delete(object key)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Remove data and keys from the external resource for the given collection of keys, if present.
+        /// <para />
+        /// The order that individual deletes occur is undefined.
+        /// <para />
+        /// If this operation fails (by throwing an exception) after a partial success,
+        /// the writer must remove any successfully written entries from the entries collection
+        /// so that the caching implementation knows what succeeded and can mutate the cache.
+        /// <para />
+        /// Expiry of a cache entry is not a delete hence will not cause this method to be invoked.
+        /// <para />
+        /// This method may include keys even if there is no mapping for that key,
+        /// in which case the data represented by that key should be removed from the underlying resource.
+        /// </summary>
+        /// <param name="keys">a mutable collection of keys for entries to delete. Upon invocation,
+        /// it contains the keys to delete for write-through. Upon return the collection must only contain
+        /// the keys that were not successfully deleted.</param>
+        public virtual void DeleteAll(ICollection keys)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Tells store to commit or rollback a transaction depending on the value of the
+        /// <c>commit</c> parameter.
+        /// </summary>
+        /// <param name="commit"><c>True</c> if transaction should commit, <c>false</c> for rollback.</param>
+        public virtual void SessionEnd(bool commit)
+        {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Store/CacheStoreAdapter.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Store/CacheStoreAdapter.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Store/CacheStoreAdapter.cs
new file mode 100644
index 0000000..1930d0c
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Store/CacheStoreAdapter.cs
@@ -0,0 +1,146 @@
+/*
+ * 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.Cache.Store
+{
+    using System;
+    using System.Collections;
+    using System.Linq;
+
+    /// <summary>
+    /// Cache storage convenience adapter. It provides default implementation for 
+    /// bulk operations, such as <code>LoadAll</code>, <code>PutAll</code> and
+    /// <code>RemoveAll</code> by sequentially calling corresponding <code>Load</code>,
+    /// <code>Put</code> and <code>Remove</code> operations. Use this adapter whenever 
+    /// such behaviour is acceptable. However in many cases it maybe more preferable 
+    /// to take advantage of database batch update functionality, and therefore default 
+    /// adapter implementation may not be the best option.
+    /// <para/>
+    /// Note that <code>LoadCache</code> method has empty implementation because it is 
+    /// essentially up to the user to invoke it with specific arguments.
+    /// </summary>
+    public abstract class CacheStoreAdapter : ICacheStore
+    {
+        /// <summary>
+        /// Loads all values from underlying persistent storage. Note that keys are
+        /// not passed, so it is up to implementation to figure out what to load.
+        /// This method is called whenever <see cref="ICache{K,V}.LocalLoadCache" />
+        /// method is invoked which is usually to preload the cache from persistent storage.
+        /// <para />
+        /// This method is optional, and cache implementation
+        /// does not depend on this method to do anything.
+        /// <para />
+        /// For every loaded value method provided action should be called.
+        /// The action will then make sure that the loaded value is stored in cache.
+        /// </summary>
+        /// <param name="act">Action for loaded values.</param>
+        /// <param name="args">Optional arguemnts passed to <see cref="ICache{K,V}.LocalLoadCache" /> method.</param>
+        public virtual void LoadCache(Action<object, object> act, params object[] args)
+        {
+            // No-op.
+        }
+        
+        /// <summary>
+        /// Loads multiple objects. Application developers should implement this method to customize
+        /// the loading of cache entries. This method is called when the requested object is not in the cache.
+        /// If an object can't be loaded, it is not returned in the resulting map.
+        /// </summary>
+        /// <param name="keys">Keys identifying the values to be loaded.</param>
+        /// <returns>
+        /// A map of key, values to be stored in the cache.
+        /// </returns>
+        public virtual IDictionary LoadAll(ICollection keys)
+        {
+            return keys.OfType<object>().ToDictionary(key => key, Load);
+        }
+        
+        /// <summary>
+        /// Writes all.
+        /// </summary>
+        /// <param name="entries">The map.</param>
+        public virtual void WriteAll(IDictionary entries)
+        {
+            foreach (DictionaryEntry entry in entries)
+                Write(entry.Key, entry.Value);
+        }
+        
+        /// <summary>
+        /// Remove data and keys from the external resource for the given collection of keys, if present.
+        /// <para />
+        /// The order that individual deletes occur is undefined.
+        /// <para />
+        /// If this operation fails (by throwing an exception) after a partial success,
+        /// the writer must remove any successfully written entries from the entries collection
+        /// so that the caching implementation knows what succeeded and can mutate the cache.
+        /// <para />
+        /// Expiry of a cache entry is not a delete hence will not cause this method to be invoked.
+        /// <para />
+        /// This method may include keys even if there is no mapping for that key,
+        /// in which case the data represented by that key should be removed from the underlying resource.
+        /// </summary>
+        /// <param name="keys">a mutable collection of keys for entries to delete. Upon invocation,
+        /// it contains the keys to delete for write-through. Upon return the collection must only contain
+        /// the keys that were not successfully deleted.</param>
+        public virtual void DeleteAll(ICollection keys)
+        {
+            foreach (object key in keys)
+                Delete(key);
+        }
+        
+        /// <summary>
+        /// Tells store to commit or rollback a transaction depending on the value of the
+        /// <c>commit</c> parameter.
+        /// </summary>
+        /// <param name="commit"><c>True</c> if transaction should commit, <c>false</c> for rollback.</param>
+        public virtual void SessionEnd(bool commit)
+        {
+            // No-op.
+        }
+        
+        /// <summary>
+        /// Loads an object. Application developers should implement this method to customize the loading
+        /// of a value for a cache entry.
+        /// This method is called by a cache when a requested entry is not in the cache.
+        /// If the object can't be loaded <code>null</code> should be returned.
+        /// </summary>
+        /// <param name="key">The key identifying the object being loaded.</param>
+        /// <returns>
+        /// The value for the entry that is to be stored in the cache
+        /// or <code>null</code> if the object can't be loaded
+        /// </returns>
+        public abstract object Load(object key);
+
+        /// <summary>
+        /// Write the specified value under the specified key to the external resource.
+        /// <para />
+        /// This method is intended to support both key/value creation and value update.
+        /// </summary>
+        /// <param name="key">Key to write.</param>
+        /// <param name="val">Value to write.</param>
+        public abstract void Write(object key, object val);
+        
+        /// <summary>
+        /// Delete the cache entry from the external resource.
+        /// <para />
+        /// Expiry of a cache entry is not a delete hence will not cause this method to be invoked.
+        /// <para />
+        /// This method is invoked even if no mapping for the key exists.
+        /// </summary>
+        /// <param name="key">The key that is used for the delete operation.</param>
+        public abstract void Delete(object key);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Store/CacheStoreException.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Store/CacheStoreException.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Store/CacheStoreException.cs
new file mode 100644
index 0000000..f5f398b
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Store/CacheStoreException.cs
@@ -0,0 +1,66 @@
+/*
+ * 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.Cache.Store
+{
+    using System;
+    using System.Runtime.Serialization;
+
+    /// <summary>
+    /// Indicates an error during CacheStore operation.
+    /// </summary>
+    [Serializable]
+    public class CacheStoreException : CacheException
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheStoreException"/> class.
+        /// </summary>
+        public CacheStoreException()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheStoreException"/> class.
+        /// </summary>
+        /// <param name="message">The message that describes the error.</param>
+        public CacheStoreException(string message) : base(message)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheStoreException"/> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        public CacheStoreException(string message, Exception cause) : base(message, cause)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheStoreException"/> class.
+        /// </summary>
+        /// <param name="info">Serialization information.</param>
+        /// <param name="ctx">Streaming context.</param>
+        protected CacheStoreException(SerializationInfo info, StreamingContext ctx) : base(info, ctx)
+        {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Store/ICacheStore.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Store/ICacheStore.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Store/ICacheStore.cs
new file mode 100644
index 0000000..4660dab
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Store/ICacheStore.cs
@@ -0,0 +1,184 @@
+/*
+ * 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.Cache.Store
+{
+    using System;
+    using System.Collections;
+    using Apache.Ignite.Core.Transactions;
+
+    /// <summary>
+    /// API for cache persistent storage for read-through and write-through behavior.
+    ///
+    /// Persistent store is configured in Ignite's Spring XML configuration file via
+    /// <c>CacheConfiguration.setStore()</c> property. If you have an implementation
+    /// of cache store in .NET, you should use special Java wrapper which accepts assembly name and
+    /// class name of .NET store implementation (both properties are mandatory).
+    /// 
+    /// Optionally, you may specify "properies" property to set any property values on an instance of your store.
+    /// <example>
+    /// Here is an example:
+    /// <code>
+    /// <bean class="org.apache.ignite.configuration.CacheConfiguration">
+    ///     ...
+    ///     <property name="cacheStoreFactory">
+    ///         <bean class="org.apache.ignite.platform.dotnet.PlatformDotNetCacheStoreFactory">
+    ///             <property name="assemblyName" value="MyAssembly"/>
+    ///             <property name="className" value="MyApp.MyCacheStore"/>
+    ///             <property name="properties">
+    ///                 <map>
+    ///                     <entry key="IntProperty">
+    ///                         <value type="java.lang.Integer">42</value>
+    ///                     </entry>
+    ///                     <entry key="StringProperty" value="String value"/>
+    ///                 </map>
+    ///             </property>
+    ///         </bean>
+    ///     </property>
+    ///     ...
+    /// </bean>
+    /// </code>
+    /// </example>
+    /// Assemply name and class name are passed to <a target="_blank" href="http://msdn.microsoft.com/en-us/library/d133hta4.aspx"><b>System.Activator.CreateInstance(String, String)</b></a>
+    /// method during node startup to create an instance of cache store. Refer to its documentation for details.
+    /// <para/>
+    /// All transactional operations of this API are provided with ongoing <see cref="ITransaction"/>,
+    /// if any. You can attach any metadata to transaction, e.g. to recognize if several operations 
+    /// belong to the same transaction or not.
+    /// <example>
+    /// Here is an example of how attach a ODBC connection as transaction metadata:
+    /// <code>
+    /// OdbcConnection conn = tx.Meta("some.name");
+    ///
+    /// if (conn == null)
+    /// {
+    ///     conn = ...; // Create or get connection.
+    ///
+    ///     // Store connection in transaction metadata, so it can be accessed
+    ///     // for other operations on the same transaction.
+    ///     tx.AddMeta("some.name", conn);
+    /// }
+    /// </code>
+    /// </example>
+    /// </summary>
+    public interface ICacheStore
+    {
+        /// <summary>
+        /// Loads all values from underlying persistent storage. Note that keys are
+        /// not passed, so it is up to implementation to figure out what to load.
+        /// This method is called whenever <see cref="ICache{K,V}.LocalLoadCache"/>
+        /// method is invoked which is usually to preload the cache from persistent storage.
+        /// <para/>
+        /// This method is optional, and cache implementation
+        /// does not depend on this method to do anything.
+        /// <para/>
+        /// For every loaded value method provided action should be called.
+        /// The action will then make sure that the loaded value is stored in cache.
+        /// </summary>
+        /// <param name="act">Action for loaded values.</param>
+        /// <param name="args">Optional arguemnts passed to <see cref="ICache{K,V}.LocalLoadCache"/> method.</param>
+        /// <exception cref="CacheStoreException" />
+        void LoadCache(Action<object, object> act, params object[] args);
+
+        /// <summary>
+        /// Loads an object. Application developers should implement this method to customize the loading 
+        /// of a value for a cache entry. 
+        /// This method is called by a cache when a requested entry is not in the cache. 
+        /// If the object can't be loaded <code>null</code> should be returned.
+        /// </summary>
+        /// <param name="key">The key identifying the object being loaded.</param>
+        /// <returns>The value for the entry that is to be stored in the cache 
+        /// or <code>null</code> if the object can't be loaded</returns>
+        /// <exception cref="CacheStoreException" />
+        object Load(object key);
+
+        /// <summary>
+        /// Loads multiple objects. Application developers should implement this method to customize 
+        /// the loading of cache entries. This method is called when the requested object is not in the cache. 
+        /// If an object can't be loaded, it is not returned in the resulting map.
+        /// </summary>
+        /// <param name="keys">Keys identifying the values to be loaded.</param>
+        /// <returns>A map of key, values to be stored in the cache.</returns>
+        /// <exception cref="CacheStoreException" />
+        IDictionary LoadAll(ICollection keys);
+
+        /// <summary>
+        /// Write the specified value under the specified key to the external resource.
+        /// <para />
+        /// This method is intended to support both key/value creation and value update.
+        /// </summary>
+        /// <param name="key">Key to write.</param>
+        /// <param name="val">Value to write.</param>
+        /// <exception cref="CacheStoreException" />
+        void Write(object key, object val);
+
+        /// <summary>
+        /// Write the specified entries to the external resource. 
+        /// This method is intended to support both insert and update.
+        /// <para />
+        /// The order that individual writes occur is undefined.
+        /// <para />
+        /// If this operation fails (by throwing an exception) after a partial success, 
+        /// the writer must remove any successfully written entries from the entries collection 
+        /// so that the caching implementation knows what succeeded and can mutate the cache.
+        /// </summary>
+        /// <param name="entries">a mutable collection to write. Upon invocation,  it contains the entries 
+        /// to write for write-through. Upon return the collection must only contain entries 
+        /// that were not successfully written. (see partial success above).</param>
+        /// <exception cref="CacheStoreException" />
+        void WriteAll(IDictionary entries);
+
+        /// <summary>
+        /// Delete the cache entry from the external resource.
+        /// <para />
+        /// Expiry of a cache entry is not a delete hence will not cause this method to be invoked.
+        /// <para />
+        /// This method is invoked even if no mapping for the key exists.
+        /// </summary>
+        /// <param name="key">The key that is used for the delete operation.</param>
+        /// <exception cref="CacheStoreException" />
+        void Delete(object key);
+
+        /// <summary>
+        /// Remove data and keys from the external resource for the given collection of keys, if present.
+        /// <para />
+        /// The order that individual deletes occur is undefined.
+        /// <para />
+        /// If this operation fails (by throwing an exception) after a partial success, 
+        /// the writer must remove any successfully written entries from the entries collection 
+        /// so that the caching implementation knows what succeeded and can mutate the cache.
+        /// <para />
+        /// Expiry of a cache entry is not a delete hence will not cause this method to be invoked.
+        /// <para />
+        /// This method may include keys even if there is no mapping for that key, 
+        /// in which case the data represented by that key should be removed from the underlying resource.
+        /// </summary>
+        /// <param name="keys">a mutable collection of keys for entries to delete. Upon invocation, 
+        /// it contains the keys to delete for write-through. Upon return the collection must only contain 
+        /// the keys that were not successfully deleted.</param>
+        /// <exception cref="CacheStoreException" />
+        void DeleteAll(ICollection keys);
+
+        /// <summary>
+        /// Tells store to commit or rollback a transaction depending on the value of the
+        /// <c>commit</c> parameter.
+        /// </summary>
+        /// <param name="commit"><c>True</c> if transaction should commit, <c>false</c> for rollback.</param>
+        /// <exception cref="CacheStoreException" />
+        void SessionEnd(bool commit);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Store/ICacheStoreSession.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Store/ICacheStoreSession.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Store/ICacheStoreSession.cs
new file mode 100644
index 0000000..e20a660
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Store/ICacheStoreSession.cs
@@ -0,0 +1,42 @@
+/*
+ * 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.Cache.Store
+{
+    using System.Collections.Generic;
+
+    /// <summary>
+    /// Session for the cache store operations. The main purpose of cache store session
+    /// is to hold context between multiple store invocations whenever in transaction. For example,
+    /// you can save current database connection in the session <see cref="Properties"/> map. You can then
+    /// commit this connection in the <see cref="ICacheStore.SessionEnd(bool)"/> method.
+    /// </summary>
+    public interface ICacheStoreSession
+    {
+        /// <summary>
+        /// Cache name for the current store operation. Note that if the same store
+        /// is reused between different caches, then the cache name will change between
+        /// different store operations.
+        /// </summary>
+        string CacheName { get; }
+
+        /// <summary>
+        /// Current session properties. You can add properties directly to the returned map.
+        /// </summary>
+        IDictionary<object, object> Properties { get; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/ClusterGroupEmptyException.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/ClusterGroupEmptyException.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/ClusterGroupEmptyException.cs
new file mode 100644
index 0000000..81e4a56
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/ClusterGroupEmptyException.cs
@@ -0,0 +1,70 @@
+/*
+ * 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.Cluster 
+{
+    using System;
+    using System.Runtime.Serialization;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// Indicates an illegal call on empty projection. Thrown by projection when operation
+    /// that requires at least one node is called on empty projection.
+    /// </summary>
+    [Serializable]
+    public class ClusterGroupEmptyException : IgniteException
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ClusterGroupEmptyException"/> class.
+        /// </summary>
+        public ClusterGroupEmptyException()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ClusterGroupEmptyException"/> class.
+        /// </summary>
+        /// <param name="msg">Exception message.</param>
+        public ClusterGroupEmptyException(string msg) : base(msg)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ClusterGroupEmptyException"/> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        public ClusterGroupEmptyException(string message, Exception cause)
+            : base(message, cause)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ClusterGroupEmptyException"/> class.
+        /// </summary>
+        /// <param name="info">Serialization info.</param>
+        /// <param name="ctx">Streaming context.</param>
+        protected ClusterGroupEmptyException(SerializationInfo info, StreamingContext ctx)
+            : base(info, ctx)
+        {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/ClusterTopologyException.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/ClusterTopologyException.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/ClusterTopologyException.cs
new file mode 100644
index 0000000..ba30f51
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/ClusterTopologyException.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.Cluster
+{
+    using System;
+    using System.Runtime.Serialization;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// Indicates an error with grid topology (e.g., crashed node, etc.)
+    /// </summary>
+    [Serializable]
+    public class ClusterTopologyException : IgniteException
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ClusterTopologyException"/> class.
+        /// </summary>
+        public ClusterTopologyException()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ClusterTopologyException"/> class.
+        /// </summary>
+        /// <param name="msg">Exception message.</param>
+        public ClusterTopologyException(string msg) : base(msg)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ClusterTopologyException"/> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        public ClusterTopologyException(string message, Exception cause)
+            : base(message, cause)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ClusterTopologyException"/> class.
+        /// </summary>
+        /// <param name="info">Serialization info.</param>
+        /// <param name="ctx">Streaming context.</param>
+        protected ClusterTopologyException(SerializationInfo info, StreamingContext ctx)
+            : base(info, ctx)
+        {
+            // No-op.
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/ICluster.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/ICluster.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/ICluster.cs
new file mode 100644
index 0000000..405375e
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/ICluster.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.Cluster
+{
+    using System;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// Represents whole cluster (group of all nodes in a cluster).
+    /// <para/>
+    /// All members are thread-safe and may be used concurrently from multiple threads.
+    /// </summary>
+    public interface ICluster : IClusterGroup
+    {
+        /// <summary>
+        /// Gets monadic projection consisting from the local node.
+        /// </summary>
+        /// <returns>Monadic projection consisting from the local node.</returns>
+        IClusterGroup ForLocal();
+
+        /// <summary>
+        /// Gets local Ignite node.
+        /// </summary>
+        /// <returns>Local Ignite node.</returns>
+        IClusterNode LocalNode
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Pings a remote node.
+        /// </summary>
+        /// <param name="nodeId">ID of a node to ping.</param>
+        /// <returns>True if node for a given ID is alive, false otherwise.</returns>
+        bool PingNode(Guid nodeId);
+
+        /// <summary>
+        /// Gets current topology version. In case of TCP discovery topology versions are sequential 
+        /// - they start from 1 and get incremented every time whenever a node joins or leaves. 
+        /// For other discovery SPIs topology versions may not be (and likely are not) sequential.
+        /// </summary>
+        /// <value>
+        /// Current topology version.
+        /// </value>
+        long TopologyVersion { get; }
+
+        /// <summary>
+        /// Gets a topology by version. Returns null if topology history storage doesn't contain 
+        /// specified topology version (history currently keeps the last 1000 snapshots).
+        /// </summary>
+        /// <param name="ver">Topology version.</param>
+        /// <returns>Collection of Ignite nodes which represented by specified topology version, 
+        /// if it is present in history storage, null otherwise.</returns>
+        /// <exception cref="IgniteException">If underlying SPI implementation does not support 
+        /// topology history. Currently only <code>org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi</code>
+        /// supports topology history.</exception>
+        ICollection<IClusterNode> Topology(long ver);
+
+        /// <summary>
+        /// Resets local I/O, job, and task execution metrics.
+        /// </summary>
+        void ResetMetrics();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/IClusterGroup.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/IClusterGroup.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/IClusterGroup.cs
new file mode 100644
index 0000000..1fa11b5
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/IClusterGroup.cs
@@ -0,0 +1,229 @@
+/*
+ * 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.Cluster 
+{
+    using System;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Events;
+    using Apache.Ignite.Core.Messaging;
+    using Apache.Ignite.Core.Services;
+
+    /// <summary>
+    /// Defines grid projection which represents a common functionality over a group of nodes.
+    /// Grid projection allows to group Ignite nodes into various subgroups to perform distributed
+    /// operations on them. All ForXXX(...)' methods will create a child grid projection
+    /// from existing projection. If you create a new projection from current one, then the resulting
+    /// projection will include a subset of nodes from current projection. The following code snippet
+    /// shows how to create grid projections:
+    /// <code>
+    /// var g = Ignition.GetIgnite();
+    /// 
+    /// // Projection over remote nodes.
+    /// var remoteNodes = g.ForRemotes();
+    /// 
+    /// // Projection over random remote node.
+    /// var randomNode = g.ForRandom();
+    /// 
+    /// // Projection over all nodes with cache named "myCache" enabled.
+    /// var cacheNodes = g.ForCacheNodes("myCache");
+    /// 
+    /// // Projection over all nodes that have user attribute "group" set to value "worker".
+    /// var workerNodes = g.ForAttribute("group", "worker");
+    /// </code>
+    /// Grid projection provides functionality for executing tasks and closures over 
+    /// nodes in this projection using <see cref="IClusterGroup.Compute()"/>.
+    /// <para/>
+    /// All members are thread-safe and may be used concurrently from multiple threads.
+    /// </summary>
+    public interface IClusterGroup {
+        /// <summary>
+        /// Instance of grid.
+        /// </summary>
+        IIgnite Ignite
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Gets compute functionality over this grid projection. All operations
+        /// on the returned ICompute instance will only include nodes from
+        /// this projection.
+        /// </summary>
+        /// <returns>Compute instance over this grid projection.</returns>
+        ICompute Compute();
+
+        /// <summary>
+        /// Creates a grid projection over a given set of nodes.
+        /// </summary>
+        /// <param name="nodes">Collection of nodes to create a projection from.</param>
+        /// <returns>Projection over provided Ignite nodes.</returns>
+        IClusterGroup ForNodes(IEnumerable<IClusterNode> nodes);
+
+        /// <summary>
+        /// Creates a grid projection over a given set of nodes.
+        /// </summary>
+        /// <param name="nodes">Collection of nodes to create a projection from.</param>
+        /// <returns>Projection over provided Ignite nodes.</returns>
+        IClusterGroup ForNodes(params IClusterNode[] nodes);
+
+        /// <summary>
+        /// Creates a grid projection over a given set of node IDs.
+        /// </summary>
+        /// <param name="ids">Collection of node IDs to create a projection from.</param>
+        /// <returns>Projection over provided Ignite node IDs.</returns>
+        IClusterGroup ForNodeIds(IEnumerable<Guid> ids);
+
+        /// <summary>
+        /// Creates a grid projection over a given set of node IDs.
+        /// </summary>
+        /// <param name="ids">Collection of node IDs to create a projection from.</param>
+        /// <returns>Projection over provided Ignite node IDs.</returns>
+        IClusterGroup ForNodeIds(params Guid[] ids);
+
+        /// <summary>
+        /// Creates a grid projection which includes all nodes that pass the given predicate filter.
+        /// </summary>
+        /// <param name="p">Predicate filter for nodes to include into this projection.</param>
+        /// <returns>Grid projection for nodes that passed the predicate filter.</returns>
+        IClusterGroup ForPredicate(Func<IClusterNode, bool> p);
+
+        /// <summary>
+        /// Creates projection for nodes containing given name and value
+        /// specified in user attributes.
+        /// </summary>
+        /// <param name="name">Name of the attribute.</param>
+        /// <param name="val">Optional attribute value to match.</param>
+        /// <returns>Grid projection for nodes containing specified attribute.</returns>
+        IClusterGroup ForAttribute(string name, string val);
+
+        /// <summary>
+        /// Creates projection for all nodes that have cache with specified name running.
+        /// </summary>
+        /// <param name="name">Cache name to include into projection.</param>
+        /// <returns>Projection over nodes that have specified cache running.</returns>
+        IClusterGroup ForCacheNodes(string name);
+        
+        /// <summary>
+        /// Creates projection for all nodes that have cache with specified name running 
+        /// and cache distribution mode is PARTITIONED_ONLY or NEAR_PARTITIONED.
+        /// </summary>
+        /// <param name="name">Cache name to include into projection.</param>
+        /// <returns>Projection over nodes that have specified cache running.</returns>
+        IClusterGroup ForDataNodes(string name);
+        
+        /// <summary>
+        /// Creates projection for all nodes that have cache with specified name running 
+        /// and cache distribution mode is CLIENT_ONLY or NEAR_ONLY.
+        /// </summary>
+        /// <param name="name">Cache name to include into projection.</param>
+        /// <returns>Projection over nodes that have specified cache running.</returns>
+        IClusterGroup ForClientNodes(string name);
+
+        /// <summary>
+        /// Gets grid projection consisting from the nodes in this projection excluding the local node.
+        /// </summary>
+        /// <returns>Grid projection consisting from the nodes in this projection excluding the local node.</returns>
+        IClusterGroup ForRemotes();
+
+        /// <summary>
+        /// Gets grid projection consisting from the nodes in this projection residing on the
+        /// same host as given node.
+        /// </summary>
+        /// <param name="node">Node residing on the host for which projection is created.</param>
+        /// <returns>Projection for nodes residing on the same host as passed in node.</returns>
+        IClusterGroup ForHost(IClusterNode node);
+
+        /// <summary>
+        /// Creates grid projection with one random node from current projection.
+        /// </summary>
+        /// <returns>Grid projection with one random node from current projection.</returns>
+        IClusterGroup ForRandom();
+
+        /// <summary>
+        /// Creates grid projection with one oldest node in the current projection.
+        /// The resulting projection is dynamic and will always pick the next oldest
+        /// node if the previous one leaves topology even after the projection has
+        /// been created.
+        /// </summary>
+        /// <returns>Grid projection with one oldest node from the current projection.</returns>
+        IClusterGroup ForOldest();
+
+        /// <summary>
+        /// Creates grid projection with one youngest node in the current projection.
+        /// The resulting projection is dynamic and will always pick the newest
+        /// node in the topology, even if more nodes entered after the projection
+        /// has been created.
+        /// </summary>
+        /// <returns>Grid projection with one youngest node from the current projection.</returns>
+        IClusterGroup ForYoungest();
+
+        /// <summary>
+        /// Creates grid projection for nodes supporting .Net, i.e. for nodes started with Ignite.exe.
+        /// </summary>
+        /// <returns>Grid projection for nodes supporting .Net.</returns>
+        IClusterGroup ForDotNet();
+
+        /// <summary>
+        /// Gets read-only collections of nodes in this projection.
+        /// </summary>
+        /// <returns>All nodes in this projection.</returns>
+        ICollection<IClusterNode> Nodes();
+
+        /// <summary>
+        /// Gets a node for given ID from this grid projection.
+        /// </summary>
+        /// <param name="id">Node ID.</param>
+        /// <returns>Node with given ID from this projection or null if such node does not 
+        /// exist in this projection.</returns>
+        IClusterNode Node(Guid id);
+
+        /// <summary>
+        /// Gets first node from the list of nodes in this projection.
+        /// </summary>
+        /// <returns>Node.</returns>
+        IClusterNode Node();
+
+        /// <summary>
+        /// Gets a metrics snapshot for this projection
+        /// </summary>
+        /// <returns>Grid projection metrics snapshot.</returns>
+        IClusterMetrics Metrics();
+
+        /// <summary>
+        /// Gets messaging facade over nodes within this cluster group.  All operations on the returned 
+        /// <see cref="IMessaging"/>> instance will only include nodes from current cluster group.
+        /// </summary>
+        /// <returns>Messaging instance over this cluster group.</returns>
+        IMessaging Message();
+
+        /// <summary>
+        /// Gets events facade over nodes within this cluster group.  All operations on the returned 
+        /// <see cref="IEvents"/>> instance will only include nodes from current cluster group.
+        /// </summary>
+        /// <returns>Events instance over this cluster group.</returns>
+        IEvents Events();
+
+        /// <summary>
+        /// Gets services facade over nodes within this cluster group.  All operations on the returned 
+        /// <see cref="IServices"/>> instance will only include nodes from current cluster group.
+        /// </summary>
+        /// <returns>Services instance over this cluster group.</returns>
+        IServices Services();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/IClusterMetrics.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/IClusterMetrics.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/IClusterMetrics.cs
new file mode 100644
index 0000000..24f0249
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/IClusterMetrics.cs
@@ -0,0 +1,515 @@
+/*
+ * 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.Cluster
+{
+    using System;
+
+    /// <summary>
+    /// Represents runtime information of a cluster. Apart from obvious
+    /// statistical value, this information is used for implementation of
+    /// load balancing, failover, and collision SPIs. For example, collision SPI
+    /// in combination with fail-over SPI could check if other nodes don't have
+    /// any active or waiting jobs and fail-over some jobs to those nodes.
+    /// <para />
+    /// Node metrics for any node can be accessed via <see cref="IClusterNode.Metrics()"/> 
+    /// method. Keep in mind that there will be a certain network delay (usually
+    /// equal to heartbeat delay) for the accuracy of node metrics. However, when accessing
+    /// metrics on local node the metrics are always accurate and up to date.
+    /// </summary>
+    public interface IClusterMetrics
+    {
+        /// <summary>
+        /// Last update time of this node metrics.
+        /// </summary>
+        DateTime LastUpdateTime
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Maximum number of jobs that ever ran concurrently on this node.
+        /// </summary>
+        int MaximumActiveJobs
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Number of currently active jobs concurrently executing on the node.
+        /// </summary>
+        int CurrentActiveJobs
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Average number of active jobs. 
+        /// </summary>
+        float AverageActiveJobs
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Maximum number of waiting jobs.
+        /// </summary>
+        int MaximumWaitingJobs
+        {
+            get;
+        }
+        
+        /// <summary>
+        /// Number of queued jobs currently waiting to be executed.
+        /// </summary>
+        int CurrentWaitingJobs
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Average number of waiting jobs.
+        /// </summary>
+        float AverageWaitingJobs
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Maximum number of jobs rejected at once.
+        /// </summary>
+        int MaximumRejectedJobs
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Number of jobs rejected after more recent collision resolution operation.
+        /// </summary>
+        int CurrentRejectedJobs
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Average number of jobs this node rejects during collision resolution operations.
+        /// </summary>
+        float AverageRejectedJobs
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Total number of jobs this node rejects during collision resolution operations since node startup.
+        /// </summary>
+        int TotalRejectedJobs
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Maximum number of cancelled jobs ever had running concurrently.
+        /// </summary>
+        int MaximumCancelledJobs
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Number of cancelled jobs that are still running.
+        /// </summary>
+        int CurrentCancelledJobs
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Average number of cancelled jobs.
+        /// </summary>
+        float AverageCancelledJobs
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Total number of cancelled jobs since node startup.
+        /// </summary>
+        int TotalCancelledJobs
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Total number of jobs handled by the node since node startup.
+        /// </summary>
+        int TotalExecutedJobs
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Maximum time a job ever spent waiting in a queue to be executed.
+        /// </summary>
+        long MaximumJobWaitTime
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Current time an oldest jobs has spent waiting to be executed.
+        /// </summary>
+        long CurrentJobWaitTime
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Average time jobs spend waiting in the queue to be executed.
+        /// </summary>
+        double AverageJobWaitTime
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Time it took to execute the longest job on the node.
+        /// </summary>
+        long MaximumJobExecuteTime
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Longest time a current job has been executing for.
+        /// </summary>
+        long CurrentJobExecuteTime
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Average job execution time.
+        /// </summary>
+        double AverageJobExecuteTime
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Total number of jobs handled by the node. 
+        /// </summary>
+        int TotalExecutedTasks
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Total time this node spent executing jobs.
+        /// </summary>
+        long TotalBusyTime
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Total time this node spent idling.
+        /// </summary>
+        long TotalIdleTime
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Time this node spend idling since executing last job.
+        /// </summary>
+        long CurrentIdleTime
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Percentage of time this node is busy.
+        /// </summary>
+        float BusyTimePercentage
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Percentage of time this node is idle
+        /// </summary>
+        float IdleTimePercentage
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Returns the number of CPUs available to the Java Virtual Machine.
+        /// </summary>
+        int TotalCpus
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Returns the CPU usage usage in [0, 1] range.
+        /// </summary>
+        double CurrentCpuLoad
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Average of CPU load values in [0, 1] range over all metrics kept in the history.
+        /// </summary>
+        double AverageCpuLoad
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Average time spent in CG since the last update.
+        /// </summary>
+        double CurrentGcCpuLoad
+        {
+            get;
+        }
+        
+        /// <summary>
+        /// Amount of heap memory in bytes that the JVM
+        /// initially requests from the operating system for memory management.
+        /// This method returns <code>-1</code> if the initial memory size is undefined.
+        /// <para />
+        /// This value represents a setting of the heap memory for Java VM and is
+        /// not a sum of all initial heap values for all memory pools.
+        /// </summary>
+        long HeapMemoryInitialized
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Current heap size that is used for object allocation.
+        /// The heap consists of one or more memory pools. This value is
+        /// the sum of used heap memory values of all heap memory pools.
+        /// <para />
+        /// The amount of used memory in the returned is the amount of memory
+        /// occupied by both live objects and garbage objects that have not
+        /// been collected, if any.
+        /// </summary>
+        long HeapMemoryUsed
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Amount of heap memory in bytes that is committed for the JVM to use. This amount of memory is
+        /// guaranteed for the JVM to use. The heap consists of one or more memory pools. This value is
+        /// the sum of committed heap memory values of all heap memory pools.
+        /// </summary>
+        long HeapMemoryCommitted
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Mmaximum amount of heap memory in bytes that can be used for memory management.
+        /// This method returns <code>-1</code> if the maximum memory size is undefined.
+        /// <para />
+        /// This amount of memory is not guaranteed to be available for memory management if 
+        /// it is greater than the amount of committed memory. The JVM may fail to allocate
+        /// memory even if the amount of used memory does not exceed this maximum size.
+        /// <para />
+        /// This value represents a setting of the heap memory for Java VM and is
+        /// not a sum of all initial heap values for all memory pools.
+        /// </summary>
+        long HeapMemoryMaximum
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Total amount of heap memory in bytes. This method returns <code>-1</code>
+        /// if the total memory size is undefined.
+        /// <para />
+        /// This amount of memory is not guaranteed to be available for memory management if it is 
+        /// greater than the amount of committed memory. The JVM may fail to allocate memory even 
+        /// if the amount of used memory does not exceed this maximum size.
+        /// <para />
+        /// This value represents a setting of the heap memory for Java VM and is
+        /// not a sum of all initial heap values for all memory pools.
+        /// </summary>
+        long HeapMemoryTotal
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Amount of non-heap memory in bytes that the JVM initially requests from the operating 
+        /// system for memory management.
+        /// </summary>
+        long NonHeapMemoryInitialized
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Current non-heap memory size that is used by Java VM.
+        /// </summary>
+        long NonHeapMemoryUsed
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Amount of non-heap memory in bytes that is committed for the JVM to use. 
+        /// </summary>
+        long NonHeapMemoryCommitted
+        {
+            get;
+        }
+        
+        /// <summary>
+        /// Maximum amount of non-heap memory in bytes that can be used for memory management.
+        /// </summary>
+        long NonHeapMemoryMaximum
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Total amount of non-heap memory in bytes that can be used for memory management. 
+        /// </summary>
+        long NonHeapMemoryTotal
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Uptime of the JVM in milliseconds.
+        /// </summary>
+        long UpTime
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Start time of the JVM in milliseconds.
+        /// </summary>
+        DateTime StartTime
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Start time of the Ignite node in milliseconds.
+        /// </summary>
+        DateTime NodeStartTime
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Current number of live threads.
+        /// </summary>
+        int CurrentThreadCount
+        {
+            get;
+        }
+
+        /// <summary>
+        /// The peak live thread count.
+        /// </summary>
+        int MaximumThreadCount
+        {
+            get;
+        }
+
+        /// <summary>
+        /// The total number of threads started.
+        /// </summary>
+        long TotalStartedThreadCount
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Current number of live daemon threads.
+        /// </summary>
+        int CurrentDaemonThreadCount
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Ignite assigns incremental versions to all cache operations. This property provides
+        /// the latest data version on the node.
+        /// </summary>
+        long LastDataVersion
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Sent messages count 
+        /// </summary>
+        int SentMessagesCount
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Sent bytes count.
+        /// </summary>
+        long SentBytesCount
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Received messages count.
+        /// </summary>
+        int ReceivedMessagesCount
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Received bytes count.
+        /// </summary>
+        long ReceivedBytesCount
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Outbound messages queue size.
+        /// </summary>
+        int OutboundMessagesQueueSize
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Gets total number of nodes.
+        /// </summary>
+        int TotalNodes
+        {
+            get;
+        }
+    }
+}


[18/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
new file mode 100644
index 0000000..80b33df
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
@@ -0,0 +1,1152 @@
+/*
+ * 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.Unmanaged
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Runtime.InteropServices;
+    using System.Threading;
+    using Apache.Ignite.Core.Cache.Event;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Cache;
+    using Apache.Ignite.Core.Impl.Cache.Query.Continuous;
+    using Apache.Ignite.Core.Impl.Cache.Store;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Compute;
+    using Apache.Ignite.Core.Impl.Datastream;
+    using Apache.Ignite.Core.Impl.Events;
+    using Apache.Ignite.Core.Impl.Handle;
+    using Apache.Ignite.Core.Impl.Memory;
+    using Apache.Ignite.Core.Impl.Messaging;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Impl.Resource;
+    using Apache.Ignite.Core.Impl.Services;
+    using Apache.Ignite.Core.Lifecycle;
+    using Apache.Ignite.Core.Services;
+    using UU = UnmanagedUtils;
+
+    /// <summary>
+    /// Unmanaged callbacks.
+    /// </summary>
+    [SuppressMessage("ReSharper", "UnusedMember.Local")]
+    [SuppressMessage("Microsoft.Design", "CA1001:TypesThatOwnDisposableFieldsShouldBeDisposable", 
+        Justification = "This class instance usually lives as long as the app runs.")]
+    [SuppressMessage("Microsoft.Design", "CA1049:TypesThatOwnNativeResourcesShouldBeDisposable", 
+        Justification = "This class instance usually lives as long as the app runs.")]
+    internal unsafe class UnmanagedCallbacks
+    {
+        /** Unmanaged context. */
+        private volatile UnmanagedContext _ctx;
+
+        /** Handle registry. */
+        private readonly HandleRegistry _handleRegistry = new HandleRegistry();
+        
+        /** Grid. */
+        private volatile Ignite _ignite;
+
+        /** Keep references to created delegates. */
+        // ReSharper disable once CollectionNeverQueried.Local
+        private readonly List<Delegate> _delegates = new List<Delegate>(50);
+
+        /** Initialized flag. */
+        private readonly ManualResetEventSlim _initEvent = new ManualResetEventSlim(false);
+
+        /** Actions to be called upon Ignite initialisation. */
+        private readonly List<Action<Ignite>> _initActions = new List<Action<Ignite>>();
+
+        /** GC handle to UnmanagedCallbacks instance to prevent it from being GCed. */
+        private readonly GCHandle _thisHnd;
+
+        /** Callbacks pointer. */
+        private readonly IntPtr _cbsPtr;
+
+        /** Error type: generic. */
+        private const int ErrGeneric = 1;
+
+        /** Error type: initialize. */
+        private const int ErrJvmInit = 2;
+
+        /** Error type: attach. */
+        private const int ErrJvmAttach = 3;
+
+        /** Opeartion: prepare .Net. */
+        private const int OpPrepareDotNet = 1;
+
+        private delegate long CacheStoreCreateCallbackDelegate(void* target, long memPtr);
+        private delegate int CacheStoreInvokeCallbackDelegate(void* target, long objPtr, long memPtr, void* cb);
+        private delegate void CacheStoreDestroyCallbackDelegate(void* target, long objPtr);
+        private delegate long CacheStoreSessionCreateCallbackDelegate(void* target, long storePtr);
+
+        private delegate long CacheEntryFilterCreateCallbackDelegate(void* target, long memPtr);
+        private delegate int CacheEntryFilterApplyCallbackDelegate(void* target, long objPtr, long memPtr);
+        private delegate void CacheEntryFilterDestroyCallbackDelegate(void* target, long objPtr);
+
+        private delegate void CacheInvokeCallbackDelegate(void* target, long inMemPtr, long outMemPtr);
+
+        private delegate void ComputeTaskMapCallbackDelegate(void* target, long taskPtr, long inMemPtr, long outMemPtr);
+        private delegate int ComputeTaskJobResultCallbackDelegate(void* target, long taskPtr, long jobPtr, long memPtr);
+        private delegate void ComputeTaskReduceCallbackDelegate(void* target, long taskPtr);
+        private delegate void ComputeTaskCompleteCallbackDelegate(void* target, long taskPtr, long memPtr);
+        private delegate int ComputeJobSerializeCallbackDelegate(void* target, long jobPtr, long memPtr);
+        private delegate long ComputeJobCreateCallbackDelegate(void* target, long memPtr);
+        private delegate void ComputeJobExecuteCallbackDelegate(void* target, long jobPtr, int cancel, long memPtr);
+        private delegate void ComputeJobCancelCallbackDelegate(void* target, long jobPtr);
+        private delegate void ComputeJobDestroyCallbackDelegate(void* target, long jobPtr);
+
+        private delegate void ContinuousQueryListenerApplyCallbackDelegate(void* target, long lsnrPtr, long memPtr);
+        private delegate long ContinuousQueryFilterCreateCallbackDelegate(void* target, long memPtr);
+        private delegate int ContinuousQueryFilterApplyCallbackDelegate(void* target, long filterPtr, long memPtr);
+        private delegate void ContinuousQueryFilterReleaseCallbackDelegate(void* target, long filterPtr);
+
+        private delegate void DataStreamerTopologyUpdateCallbackDelegate(void* target, long ldrPtr, long topVer, int topSize);
+        private delegate void DataStreamerStreamReceiverInvokeCallbackDelegate(void* target, long ptr, void* cache, long memPtr, byte keepPortable);
+
+        private delegate void FutureByteResultCallbackDelegate(void* target, long futPtr, int res);
+        private delegate void FutureBoolResultCallbackDelegate(void* target, long futPtr, int res);
+        private delegate void FutureShortResultCallbackDelegate(void* target, long futPtr, int res);
+        private delegate void FutureCharResultCallbackDelegate(void* target, long futPtr, int res);
+        private delegate void FutureIntResultCallbackDelegate(void* target, long futPtr, int res);
+        private delegate void FutureFloatResultCallbackDelegate(void* target, long futPtr, float res);
+        private delegate void FutureLongResultCallbackDelegate(void* target, long futPtr, long res);
+        private delegate void FutureDoubleResultCallbackDelegate(void* target, long futPtr, double res);
+        private delegate void FutureObjectResultCallbackDelegate(void* target, long futPtr, long memPtr);
+        private delegate void FutureNullResultCallbackDelegate(void* target, long futPtr);
+        private delegate void FutureErrorCallbackDelegate(void* target, long futPtr, long memPtr);
+
+        private delegate void LifecycleOnEventCallbackDelegate(void* target, long ptr, int evt);
+
+        private delegate void MemoryReallocateCallbackDelegate(void* target, long memPtr, int cap);
+
+        private delegate long MessagingFilterCreateCallbackDelegate(void* target, long memPtr);
+        private delegate int MessagingFilterApplyCallbackDelegate(void* target, long ptr, long memPtr);
+        private delegate void MessagingFilterDestroyCallbackDelegate(void* target, long ptr);
+        
+        private delegate long EventFilterCreateCallbackDelegate(void* target, long memPtr);
+        private delegate int EventFilterApplyCallbackDelegate(void* target, long ptr, long memPtr);
+        private delegate void EventFilterDestroyCallbackDelegate(void* target, long ptr);
+
+        private delegate long ServiceInitCallbackDelegate(void* target, long memPtr);
+        private delegate void ServiceExecuteCallbackDelegate(void* target, long svcPtr, long memPtr);
+        private delegate void ServiceCancelCallbackDelegate(void* target, long svcPtr, long memPtr);
+        private delegate void ServiceInvokeMethodCallbackDelegate(void* target, long svcPtr, long inMemPtr, long outMemPtr);
+
+        private delegate int СlusterNodeFilterApplyCallbackDelegate(void* target, long memPtr);
+
+        private delegate void NodeInfoCallbackDelegate(void* target, long memPtr);
+
+        private delegate void OnStartCallbackDelegate(void* target, long memPtr);
+        private delegate void OnStopCallbackDelegate(void* target);
+        
+        private delegate void ErrorCallbackDelegate(void* target, int errType, sbyte* errClsChars, int errClsCharsLen, sbyte* errMsgChars, int errMsgCharsLen, void* errData, int errDataLen);
+
+        private delegate long ExtensionCallbackInLongOutLongDelegate(void* target, int typ, long arg1);
+        private delegate long ExtensionCallbackInLongLongOutLongDelegate(void* target, int typ, long arg1, long arg2);
+
+        /// <summary>
+        /// constructor.
+        /// </summary>
+        public UnmanagedCallbacks()
+        {
+            var cbs = new UnmanagedCallbackHandlers
+            {
+                target = IntPtr.Zero.ToPointer(), // Target is not used in .Net as we rely on dynamic FP creation.
+
+                cacheStoreCreate = CreateFunctionPointer((CacheStoreCreateCallbackDelegate) CacheStoreCreate),
+                cacheStoreInvoke = CreateFunctionPointer((CacheStoreInvokeCallbackDelegate) CacheStoreInvoke),
+                cacheStoreDestroy = CreateFunctionPointer((CacheStoreDestroyCallbackDelegate) CacheStoreDestroy),
+
+                cacheStoreSessionCreate = CreateFunctionPointer((CacheStoreSessionCreateCallbackDelegate) CacheStoreSessionCreate),
+                
+                cacheEntryFilterCreate = CreateFunctionPointer((CacheEntryFilterCreateCallbackDelegate)CacheEntryFilterCreate),
+                cacheEntryFilterApply = CreateFunctionPointer((CacheEntryFilterApplyCallbackDelegate)CacheEntryFilterApply),
+                cacheEntryFilterDestroy = CreateFunctionPointer((CacheEntryFilterDestroyCallbackDelegate)CacheEntryFilterDestroy),
+
+                cacheInvoke = CreateFunctionPointer((CacheInvokeCallbackDelegate) CacheInvoke),
+
+                computeTaskMap = CreateFunctionPointer((ComputeTaskMapCallbackDelegate) ComputeTaskMap),
+                computeTaskJobResult =
+                    CreateFunctionPointer((ComputeTaskJobResultCallbackDelegate) ComputeTaskJobResult),
+                computeTaskReduce = CreateFunctionPointer((ComputeTaskReduceCallbackDelegate) ComputeTaskReduce),
+                computeTaskComplete = CreateFunctionPointer((ComputeTaskCompleteCallbackDelegate) ComputeTaskComplete),
+                computeJobSerialize = CreateFunctionPointer((ComputeJobSerializeCallbackDelegate) ComputeJobSerialize),
+                computeJobCreate = CreateFunctionPointer((ComputeJobCreateCallbackDelegate) ComputeJobCreate),
+                computeJobExecute = CreateFunctionPointer((ComputeJobExecuteCallbackDelegate) ComputeJobExecute),
+                computeJobCancel = CreateFunctionPointer((ComputeJobCancelCallbackDelegate) ComputeJobCancel),
+                computeJobDestroy = CreateFunctionPointer((ComputeJobDestroyCallbackDelegate) ComputeJobDestroy),
+                continuousQueryListenerApply =
+                    CreateFunctionPointer((ContinuousQueryListenerApplyCallbackDelegate) ContinuousQueryListenerApply),
+                continuousQueryFilterCreate =
+                    CreateFunctionPointer((ContinuousQueryFilterCreateCallbackDelegate) ContinuousQueryFilterCreate),
+                continuousQueryFilterApply =
+                    CreateFunctionPointer((ContinuousQueryFilterApplyCallbackDelegate) ContinuousQueryFilterApply),
+                continuousQueryFilterRelease =
+                    CreateFunctionPointer((ContinuousQueryFilterReleaseCallbackDelegate) ContinuousQueryFilterRelease),
+                dataStreamerTopologyUpdate =
+                    CreateFunctionPointer((DataStreamerTopologyUpdateCallbackDelegate) DataStreamerTopologyUpdate),
+                dataStreamerStreamReceiverInvoke =
+                    CreateFunctionPointer((DataStreamerStreamReceiverInvokeCallbackDelegate) DataStreamerStreamReceiverInvoke),
+                
+                futureByteResult = CreateFunctionPointer((FutureByteResultCallbackDelegate) FutureByteResult),
+                futureBoolResult = CreateFunctionPointer((FutureBoolResultCallbackDelegate) FutureBoolResult),
+                futureShortResult = CreateFunctionPointer((FutureShortResultCallbackDelegate) FutureShortResult),
+                futureCharResult = CreateFunctionPointer((FutureCharResultCallbackDelegate) FutureCharResult),
+                futureIntResult = CreateFunctionPointer((FutureIntResultCallbackDelegate) FutureIntResult),
+                futureFloatResult = CreateFunctionPointer((FutureFloatResultCallbackDelegate) FutureFloatResult),
+                futureLongResult = CreateFunctionPointer((FutureLongResultCallbackDelegate) FutureLongResult),
+                futureDoubleResult = CreateFunctionPointer((FutureDoubleResultCallbackDelegate) FutureDoubleResult),
+                futureObjectResult = CreateFunctionPointer((FutureObjectResultCallbackDelegate) FutureObjectResult),
+                futureNullResult = CreateFunctionPointer((FutureNullResultCallbackDelegate) FutureNullResult),
+                futureError = CreateFunctionPointer((FutureErrorCallbackDelegate) FutureError),
+                lifecycleOnEvent = CreateFunctionPointer((LifecycleOnEventCallbackDelegate) LifecycleOnEvent),
+                memoryReallocate = CreateFunctionPointer((MemoryReallocateCallbackDelegate) MemoryReallocate),
+                nodeInfo = CreateFunctionPointer((NodeInfoCallbackDelegate) NodeInfo),
+                
+                messagingFilterCreate = CreateFunctionPointer((MessagingFilterCreateCallbackDelegate)MessagingFilterCreate),
+                messagingFilterApply = CreateFunctionPointer((MessagingFilterApplyCallbackDelegate)MessagingFilterApply),
+                messagingFilterDestroy = CreateFunctionPointer((MessagingFilterDestroyCallbackDelegate)MessagingFilterDestroy),
+
+                eventFilterCreate = CreateFunctionPointer((EventFilterCreateCallbackDelegate)EventFilterCreate),
+                eventFilterApply = CreateFunctionPointer((EventFilterApplyCallbackDelegate)EventFilterApply),
+                eventFilterDestroy = CreateFunctionPointer((EventFilterDestroyCallbackDelegate)EventFilterDestroy),
+
+                serviceInit = CreateFunctionPointer((ServiceInitCallbackDelegate)ServiceInit),
+                serviceExecute = CreateFunctionPointer((ServiceExecuteCallbackDelegate)ServiceExecute),
+                serviceCancel = CreateFunctionPointer((ServiceCancelCallbackDelegate)ServiceCancel),
+                serviceInvokeMethod = CreateFunctionPointer((ServiceInvokeMethodCallbackDelegate)ServiceInvokeMethod),
+
+                clusterNodeFilterApply = CreateFunctionPointer((СlusterNodeFilterApplyCallbackDelegate)СlusterNodeFilterApply),
+                
+                onStart = CreateFunctionPointer((OnStartCallbackDelegate)OnStart),
+                onStop = CreateFunctionPointer((OnStopCallbackDelegate)OnStop),
+                error = CreateFunctionPointer((ErrorCallbackDelegate)Error),
+                
+                extensionCbInLongOutLong = CreateFunctionPointer((ExtensionCallbackInLongOutLongDelegate)ExtensionCallbackInLongOutLong),
+                extensionCbInLongLongOutLong = CreateFunctionPointer((ExtensionCallbackInLongLongOutLongDelegate)ExtensionCallbackInLongLongOutLong)
+            };
+
+            _cbsPtr = Marshal.AllocHGlobal(UU.HandlersSize());
+
+            Marshal.StructureToPtr(cbs, _cbsPtr, false);
+
+            _thisHnd = GCHandle.Alloc(this);
+        }
+
+        /// <summary>
+        /// Gets the handle registry.
+        /// </summary>
+        public HandleRegistry HandleRegistry
+        {
+            get { return _handleRegistry; }
+        }
+
+        #region IMPLEMENTATION: CACHE
+
+        private long CacheStoreCreate(void* target, long memPtr)
+        {
+            return SafeCall(() =>
+            {
+                var cacheStore = CacheStore.CreateInstance(memPtr, _handleRegistry);
+
+                if (_ignite != null)
+                    cacheStore.Init(_ignite);
+                else
+                {
+                    lock (_initActions)
+                    {
+                        if (_ignite != null)
+                            cacheStore.Init(_ignite);
+                        else
+                            _initActions.Add(g => cacheStore.Init(g));
+                    }
+                }
+
+                return cacheStore.Handle;
+            }, true);
+        }
+
+        private int CacheStoreInvoke(void* target, long objPtr, long memPtr, void* cb)
+        {
+            return SafeCall(() =>
+            {
+                var t = _handleRegistry.Get<CacheStore>(objPtr, true);
+
+                IUnmanagedTarget cb0 = null;
+
+                if ((long) cb != 0)
+                    cb0 = new UnmanagedNonReleaseableTarget(_ctx.NativeContext, cb);
+
+                using (PlatformMemoryStream stream = IgniteManager.Memory.Get(memPtr).Stream())
+                {
+                    return t.Invoke(stream, cb0, _ignite);
+                }
+            });
+        }
+
+        private void CacheStoreDestroy(void* target, long objPtr)
+        {
+            SafeCall(() => _ignite.HandleRegistry.Release(objPtr));
+        }
+
+        private long CacheStoreSessionCreate(void* target, long storePtr)
+        {
+            return SafeCall(() => _ignite.HandleRegistry.Allocate(new CacheStoreSession()));
+        }
+
+        private long CacheEntryFilterCreate(void* target, long memPtr)
+        {
+            return SafeCall(() => CacheEntryFilterHolder.CreateInstance(memPtr, _ignite).Handle);
+        }
+
+        private int CacheEntryFilterApply(void* target, long objPtr, long memPtr)
+        {
+            return SafeCall(() =>
+            {
+                var t = _ignite.HandleRegistry.Get<CacheEntryFilterHolder>(objPtr);
+
+                using (PlatformMemoryStream stream = IgniteManager.Memory.Get(memPtr).Stream())
+                {
+                    return t.Invoke(stream);
+                }
+            });
+        }
+
+        private void CacheEntryFilterDestroy(void* target, long objPtr)
+        {
+            SafeCall(() => _ignite.HandleRegistry.Release(objPtr));
+        }
+
+        private void CacheInvoke(void* target, long inMemPtr, long outMemPtr)
+        {
+            SafeCall(() =>
+            {
+                using (PlatformMemoryStream inStream = IgniteManager.Memory.Get(inMemPtr).Stream())
+                {
+                    var result = ReadAndRunCacheEntryProcessor(inStream, _ignite);
+
+                    using (PlatformMemoryStream outStream = IgniteManager.Memory.Get(outMemPtr).Stream())
+                    {
+                        result.Write(outStream, _ignite.Marshaller);
+
+                        outStream.SynchronizeOutput();
+                    }
+                }
+            });
+        }
+
+        /// <summary>
+        /// Reads cache entry processor and related data from stream, executes it and returns the result.
+        /// </summary>
+        /// <param name="inOutStream">Stream.</param>
+        /// <param name="grid">Grid.</param>
+        /// <returns>CacheEntryProcessor result.</returns>
+        private CacheEntryProcessorResultHolder ReadAndRunCacheEntryProcessor(IPortableStream inOutStream,
+            Ignite grid)
+        {
+            var marsh = grid.Marshaller;
+
+            var key = marsh.Unmarshal<object>(inOutStream);
+            var val = marsh.Unmarshal<object>(inOutStream);
+            var isLocal = inOutStream.ReadBool();
+
+            var holder = isLocal
+                ? _handleRegistry.Get<CacheEntryProcessorHolder>(inOutStream.ReadLong(), true)
+                : marsh.Unmarshal<CacheEntryProcessorHolder>(inOutStream);
+
+            return holder.Process(key, val, val != null, grid);
+        }
+
+        #endregion
+
+        #region IMPLEMENTATION: COMPUTE
+
+        private void ComputeTaskMap(void* target, long taskPtr, long inMemPtr, long outMemPtr)
+        {
+            SafeCall(() =>
+            {
+                using (PlatformMemoryStream inStream = IgniteManager.Memory.Get(inMemPtr).Stream())
+                {
+                    using (PlatformMemoryStream outStream = IgniteManager.Memory.Get(outMemPtr).Stream())
+                    {
+                        Task(taskPtr).Map(inStream, outStream);
+                    }
+                }
+            });
+        }
+
+        private int ComputeTaskJobResult(void* target, long taskPtr, long jobPtr, long memPtr)
+        {
+            return SafeCall(() =>
+            {
+                var task = Task(taskPtr);
+
+                if (memPtr == 0)
+                {
+                    return task.JobResultLocal(Job(jobPtr));
+                }
+                
+                using (var stream = IgniteManager.Memory.Get(memPtr).Stream())
+                {
+                    return task.JobResultRemote(Job(jobPtr), stream);
+                }
+            });
+        }
+
+        private void ComputeTaskReduce(void* target, long taskPtr)
+        {
+            SafeCall(() =>
+            {
+                var task = _handleRegistry.Get<IComputeTaskHolder>(taskPtr, true);
+
+                task.Reduce();
+            });
+        }
+
+        private void ComputeTaskComplete(void* target, long taskPtr, long memPtr)
+        {
+            SafeCall(() =>
+            {
+                var task = _handleRegistry.Get<IComputeTaskHolder>(taskPtr, true);
+
+                if (memPtr == 0)
+                    task.Complete(taskPtr);
+                else
+                {
+                    using (PlatformMemoryStream stream = IgniteManager.Memory.Get(memPtr).Stream())
+                    {
+                        task.CompleteWithError(taskPtr, stream);
+                    }
+                }
+            });
+        }
+
+        private int ComputeJobSerialize(void* target, long jobPtr, long memPtr)
+        {
+            return SafeCall(() =>
+            {
+                using (PlatformMemoryStream stream = IgniteManager.Memory.Get(memPtr).Stream())
+                {
+                    return Job(jobPtr).Serialize(stream) ? 1 : 0;
+                }
+            });
+        }
+
+        private long ComputeJobCreate(void* target, long memPtr)
+        {
+            return SafeCall(() =>
+            {
+                using (PlatformMemoryStream stream = IgniteManager.Memory.Get(memPtr).Stream())
+                {
+                    ComputeJobHolder job = ComputeJobHolder.CreateJob(_ignite, stream);
+
+                    return _handleRegistry.Allocate(job);
+                }
+            });
+        }
+
+        private void ComputeJobExecute(void* target, long jobPtr, int cancel, long memPtr)
+        {
+            SafeCall(() =>
+            {
+                var job = Job(jobPtr);
+
+                if (memPtr == 0)
+                    job.ExecuteLocal(cancel == 1);
+                else
+                {
+                    using (PlatformMemoryStream stream = IgniteManager.Memory.Get(memPtr).Stream())
+                    {
+                        job.ExecuteRemote(stream, cancel == 1);
+                    }
+                }
+            });
+        }
+
+        private void ComputeJobCancel(void* target, long jobPtr)
+        {
+            SafeCall(() =>
+            {
+                Job(jobPtr).Cancel();
+            });
+        }
+
+        private void ComputeJobDestroy(void* target, long jobPtr)
+        {
+            SafeCall(() =>
+            {
+                _handleRegistry.Release(jobPtr);
+            });
+        }
+
+        /// <summary>
+        /// Get compute task using it's GC handle pointer.
+        /// </summary>
+        /// <param name="taskPtr">Task pointer.</param>
+        /// <returns>Compute task.</returns>
+        private IComputeTaskHolder Task(long taskPtr)
+        {
+            return _handleRegistry.Get<IComputeTaskHolder>(taskPtr);
+        }
+
+        /// <summary>
+        /// Get comptue job using it's GC handle pointer.
+        /// </summary>
+        /// <param name="jobPtr">Job pointer.</param>
+        /// <returns>Compute job.</returns>
+        private ComputeJobHolder Job(long jobPtr)
+        {
+            return _handleRegistry.Get<ComputeJobHolder>(jobPtr);
+        }
+
+        #endregion
+
+        #region  IMPLEMENTATION: CONTINUOUS QUERY
+
+        private void ContinuousQueryListenerApply(void* target, long lsnrPtr, long memPtr)
+        {
+            SafeCall(() =>
+            {
+                var hnd = _handleRegistry.Get<IContinuousQueryHandleImpl>(lsnrPtr);
+
+                hnd.Apply(IgniteManager.Memory.Get(memPtr).Stream());
+            });
+        }
+
+        [SuppressMessage("ReSharper", "PossibleNullReferenceException")]
+        private long ContinuousQueryFilterCreate(void* target, long memPtr)
+        {
+            return SafeCall(() =>
+            {
+                // 1. Unmarshal filter holder.
+                IPortableStream stream = IgniteManager.Memory.Get(memPtr).Stream();
+
+                var reader = _ignite.Marshaller.StartUnmarshal(stream);
+
+                ContinuousQueryFilterHolder filterHolder = reader.ReadObject<ContinuousQueryFilterHolder>();
+
+                // 2. Create real filter from it's holder.
+                Type filterWrapperTyp = typeof(ContinuousQueryFilter<,>)
+                    .MakeGenericType(filterHolder.KeyType, filterHolder.ValueType);
+
+                Type filterTyp = typeof(ICacheEntryEventFilter<,>)
+                    .MakeGenericType(filterHolder.KeyType, filterHolder.ValueType);
+
+                var filter = (IContinuousQueryFilter)filterWrapperTyp
+                    .GetConstructor(new[] { filterTyp, typeof(bool) })
+                    .Invoke(new[] { filterHolder.Filter, filterHolder.KeepPortable });
+
+                // 3. Inject grid.
+                filter.Inject(_ignite);
+
+                // 4. Allocate GC handle.
+                return filter.Allocate();
+            });
+        }
+
+        private int ContinuousQueryFilterApply(void* target, long filterPtr, long memPtr)
+        {
+            return SafeCall(() =>
+            {
+                var holder = _handleRegistry.Get<IContinuousQueryFilter>(filterPtr);
+
+                return holder.Evaluate(IgniteManager.Memory.Get(memPtr).Stream()) ? 1 : 0;
+            });
+        }
+
+        private void ContinuousQueryFilterRelease(void* target, long filterPtr)
+        {
+            SafeCall(() =>
+            {
+                var holder = _handleRegistry.Get<IContinuousQueryFilter>(filterPtr);
+
+                holder.Release();
+            });
+        }
+        
+        #endregion
+
+        #region IMPLEMENTATION: DATA STREAMER
+
+        private void DataStreamerTopologyUpdate(void* target, long ldrPtr, long topVer, int topSize)
+        {
+            SafeCall(() =>
+            {
+                var ldrRef = _handleRegistry.Get<WeakReference>(ldrPtr);
+
+                if (ldrRef == null)
+                    return;
+
+                var ldr = ldrRef.Target as IDataStreamer;
+
+                if (ldr != null)
+                    ldr.TopologyChange(topVer, topSize);
+                else
+                    _handleRegistry.Release(ldrPtr, true);
+            });
+        }
+
+        private void DataStreamerStreamReceiverInvoke(void* target, long rcvPtr, void* cache, long memPtr, 
+            byte keepPortable)
+        {
+            SafeCall(() =>
+            {
+                var stream = IgniteManager.Memory.Get(memPtr).Stream();
+
+                var reader = _ignite.Marshaller.StartUnmarshal(stream, PortableMode.ForcePortable);
+
+                var portableReceiver = reader.ReadObject<PortableUserObject>();
+
+                var receiver = _handleRegistry.Get<StreamReceiverHolder>(rcvPtr) ??
+                    portableReceiver.Deserialize<StreamReceiverHolder>();
+
+                if (receiver != null)
+                    receiver.Receive(_ignite, new UnmanagedNonReleaseableTarget(_ctx.NativeContext, cache), stream,
+                        keepPortable != 0);
+            });
+        }
+
+        #endregion
+        
+        #region IMPLEMENTATION: FUTURES
+
+        private void FutureByteResult(void* target, long futPtr, int res)
+        {
+            SafeCall(() =>
+            {
+                ProcessFuture<byte>(futPtr, fut => { fut.OnResult((byte)res); });
+            });
+        }
+
+        private void FutureBoolResult(void* target, long futPtr, int res)
+        {
+            SafeCall(() =>
+            {
+                ProcessFuture<bool>(futPtr, fut => { fut.OnResult(res == 1); });
+            });
+        }
+
+        private void FutureShortResult(void* target, long futPtr, int res)
+        {
+            SafeCall(() =>
+            {
+                ProcessFuture<short>(futPtr, fut => { fut.OnResult((short)res); });
+            });
+        }
+
+        private void FutureCharResult(void* target, long futPtr, int res)
+        {
+            SafeCall(() =>
+            {
+                ProcessFuture<char>(futPtr, fut => { fut.OnResult((char)res); });
+            });
+        }
+
+        private void FutureIntResult(void* target, long futPtr, int res)
+        {
+            SafeCall(() =>
+            {
+                ProcessFuture<int>(futPtr, fut => { fut.OnResult(res); });
+            });
+        }
+
+        private void FutureFloatResult(void* target, long futPtr, float res)
+        {
+            SafeCall(() =>
+            {
+                ProcessFuture<float>(futPtr, fut => { fut.OnResult(res); });
+            });
+        }
+
+        private void FutureLongResult(void* target, long futPtr, long res)
+        {
+            SafeCall(() =>
+            {
+                ProcessFuture<long>(futPtr, fut => { fut.OnResult(res); });
+            });
+        }
+
+        private void FutureDoubleResult(void* target, long futPtr, double res)
+        {
+            SafeCall(() =>
+            {
+                ProcessFuture<double>(futPtr, fut => { fut.OnResult(res); });
+            });
+        }
+
+        private void FutureObjectResult(void* target, long futPtr, long memPtr)
+        {
+            SafeCall(() =>
+            {
+                ProcessFuture(futPtr, fut =>
+                {
+                    IPortableStream stream = IgniteManager.Memory.Get(memPtr).Stream();
+
+                    fut.OnResult(stream);
+                });
+            });
+        }
+
+        private void FutureNullResult(void* target, long futPtr)
+        {
+            SafeCall(() =>
+            {
+                ProcessFuture(futPtr, fut => { fut.OnNullResult(); });
+            });
+        }
+
+        private void FutureError(void* target, long futPtr, long memPtr)
+        {
+            SafeCall(() =>
+            {
+                IPortableStream stream = IgniteManager.Memory.Get(memPtr).Stream();
+
+                PortableReaderImpl reader = _ignite.Marshaller.StartUnmarshal(stream);
+
+                string errCls = reader.ReadString();
+                string errMsg = reader.ReadString();
+
+                Exception err = ExceptionUtils.GetException(errCls, errMsg, reader);
+
+                ProcessFuture(futPtr, fut => { fut.OnError(err); });
+            });
+        }
+
+        /// <summary>
+        /// Process future.
+        /// </summary>
+        /// <param name="futPtr">Future pointer.</param>
+        /// <param name="action">Action.</param>
+        private void ProcessFuture(long futPtr, Action<IFutureInternal> action)
+        {
+            try
+            {
+                action(_handleRegistry.Get<IFutureInternal>(futPtr, true));
+            }
+            finally
+            {
+                _handleRegistry.Release(futPtr);
+            }
+        }
+
+        /// <summary>
+        /// Process future.
+        /// </summary>
+        /// <param name="futPtr">Future pointer.</param>
+        /// <param name="action">Action.</param>
+        private void ProcessFuture<T>(long futPtr, Action<Future<T>> action)
+        {
+            try
+            {
+                action(_handleRegistry.Get<Future<T>>(futPtr, true));
+            }
+            finally
+            {
+                _handleRegistry.Release(futPtr);
+            }
+        }
+
+        #endregion
+
+        #region IMPLEMENTATION: LIFECYCLE
+
+        private void LifecycleOnEvent(void* target, long ptr, int evt)
+        {
+            SafeCall(() =>
+            {
+                var bean = _handleRegistry.Get<LifecycleBeanHolder>(ptr);
+
+                bean.OnLifecycleEvent((LifecycleEventType)evt);
+            }, true);
+        }
+
+        #endregion
+
+        #region IMPLEMENTATION: MESSAGING
+
+        private long MessagingFilterCreate(void* target, long memPtr)
+        {
+            return SafeCall(() =>
+            {
+                MessageFilterHolder holder = MessageFilterHolder.CreateRemote(_ignite, memPtr);
+
+                return _ignite.HandleRegistry.AllocateSafe(holder);
+            });
+        }
+
+        private int MessagingFilterApply(void* target, long ptr, long memPtr)
+        {
+            return SafeCall(() =>
+            {
+                var holder = _ignite.HandleRegistry.Get<MessageFilterHolder>(ptr, false);
+                
+                if (holder == null)
+                    return 0;
+
+                using (var stream = IgniteManager.Memory.Get(memPtr).Stream())
+                {
+                    return holder.Invoke(stream);
+                }
+            });
+        }
+
+        private void MessagingFilterDestroy(void* target, long ptr)
+        {
+            SafeCall(() =>
+            {
+                _ignite.HandleRegistry.Release(ptr);
+            });
+        }
+        
+        #endregion
+
+        #region IMPLEMENTATION: EXTENSIONS
+
+        private long ExtensionCallbackInLongOutLong(void* target, int op, long arg1)
+        {
+            throw new InvalidOperationException("Unsupported operation type: " + op);
+        }
+
+        private long ExtensionCallbackInLongLongOutLong(void* target, int op, long arg1, long arg2)
+        {
+            return SafeCall(() =>
+            {
+                switch (op)
+                {
+                    case OpPrepareDotNet:
+                        var inMem = IgniteManager.Memory.Get(arg1);
+                        var outMem = IgniteManager.Memory.Get(arg2);
+
+                        PlatformMemoryStream inStream = inMem.Stream();
+                        PlatformMemoryStream outStream = outMem.Stream();
+
+                        Ignition.OnPrepare(inStream, outStream, _handleRegistry);
+
+                        return 0;
+
+                    default:
+                        throw new InvalidOperationException("Unsupported operation type: " + op);
+                }
+            }, op == OpPrepareDotNet);
+        }
+
+        #endregion
+
+        #region IMPLEMENTATION: EVENTS
+
+        private long EventFilterCreate(void* target, long memPtr)
+        {
+            return SafeCall(() => _handleRegistry.Allocate(RemoteListenEventFilter.CreateInstance(memPtr, _ignite)));
+        }
+
+        private int EventFilterApply(void* target, long ptr, long memPtr)
+        {
+            return SafeCall(() =>
+            {
+                var holder = _ignite.HandleRegistry.Get<IInteropCallback>(ptr, false);
+
+                if (holder == null)
+                    return 0;
+
+                using (var stream = IgniteManager.Memory.Get(memPtr).Stream())
+                {
+                    return holder.Invoke(stream);
+                }
+            });
+        }
+
+        private void EventFilterDestroy(void* target, long ptr)
+        {
+            SafeCall(() =>
+            {
+                _ignite.HandleRegistry.Release(ptr);
+            });
+        }
+        
+        #endregion
+
+        #region IMPLEMENTATION: SERVICES
+
+        private long ServiceInit(void* target, long memPtr)
+        {
+            return SafeCall(() =>
+            {
+                using (var stream = IgniteManager.Memory.Get(memPtr).Stream())
+                {
+                    var reader = _ignite.Marshaller.StartUnmarshal(stream);
+
+                    bool srvKeepPortable = reader.ReadBoolean();
+                    var svc = reader.ReadObject<IService>();
+
+                    ResourceProcessor.Inject(svc, _ignite);
+
+                    svc.Init(new ServiceContext(_ignite.Marshaller.StartUnmarshal(stream, srvKeepPortable)));
+
+                    return _handleRegistry.Allocate(svc);
+                }
+            });
+        }
+
+        private void ServiceExecute(void* target, long svcPtr, long memPtr)
+        {
+            SafeCall(() =>
+            {
+                var svc = _handleRegistry.Get<IService>(svcPtr, true);
+
+                using (var stream = IgniteManager.Memory.Get(memPtr).Stream())
+                {
+                    var reader = _ignite.Marshaller.StartUnmarshal(stream);
+
+                    bool srvKeepPortable = reader.ReadBoolean();
+
+                    svc.Execute(new ServiceContext(
+                        _ignite.Marshaller.StartUnmarshal(stream, srvKeepPortable)));
+                }
+            });
+        }
+
+        private void ServiceCancel(void* target, long svcPtr, long memPtr)
+        {
+            SafeCall(() =>
+            {
+                var svc = _handleRegistry.Get<IService>(svcPtr, true);
+
+                try
+                {
+                    using (var stream = IgniteManager.Memory.Get(memPtr).Stream())
+                    {
+                        var reader = _ignite.Marshaller.StartUnmarshal(stream);
+
+                        bool srvKeepPortable = reader.ReadBoolean();
+
+                        svc.Cancel(new ServiceContext(_ignite.Marshaller.StartUnmarshal(stream, srvKeepPortable)));
+                    }
+                }
+                finally
+                {
+                    _ignite.HandleRegistry.Release(svcPtr);
+                }
+            });
+        }
+
+        private void ServiceInvokeMethod(void* target, long svcPtr, long inMemPtr, long outMemPtr)
+        {
+            SafeCall(() =>
+            {
+                using (var inStream = IgniteManager.Memory.Get(inMemPtr).Stream())
+                using (var outStream = IgniteManager.Memory.Get(outMemPtr).Stream())
+                {
+                    var svc = _handleRegistry.Get<IService>(svcPtr, true);
+
+                    string mthdName;
+                    object[] mthdArgs;
+
+                    ServiceProxySerializer.ReadProxyMethod(inStream, _ignite.Marshaller, out mthdName, out mthdArgs);
+
+                    var result = ServiceProxyInvoker.InvokeServiceMethod(svc, mthdName, mthdArgs);
+
+                    ServiceProxySerializer.WriteInvocationResult(outStream, _ignite.Marshaller, result.Key, result.Value);
+
+                    outStream.SynchronizeOutput();
+                }
+            });
+        }
+
+        private int СlusterNodeFilterApply(void* target, long memPtr)
+        {
+            return SafeCall(() =>
+            {
+                using (var stream = IgniteManager.Memory.Get(memPtr).Stream())
+                {
+                    var reader = _ignite.Marshaller.StartUnmarshal(stream);
+
+                    var filter = (IClusterNodeFilter) reader.ReadObject<PortableOrSerializableObjectHolder>().Item;
+
+                    return filter.Invoke(_ignite.GetNode(reader.ReadGuid())) ? 1 : 0;
+                }
+            });
+        }
+
+        #endregion
+
+        #region IMPLEMENTATION: MISCELLANEOUS
+
+        private void NodeInfo(void* target, long memPtr)
+        {
+            SafeCall(() => _ignite.UpdateNodeInfo(memPtr));
+        }
+
+        private void MemoryReallocate(void* target, long memPtr, int cap)
+        {
+            SafeCall(() =>
+            {
+                IgniteManager.Memory.Get(memPtr).Reallocate(cap);
+            }, true);
+        }
+
+        private void OnStart(void* target, long memPtr)
+        {
+            SafeCall(() =>
+            {
+                Ignition.OnStart(IgniteManager.Memory.Get(memPtr).Stream());
+            }, true);
+        }
+
+        private void OnStop(void* target)
+        {
+            Marshal.FreeHGlobal(_cbsPtr);
+
+            // ReSharper disable once ImpureMethodCallOnReadonlyValueField
+            _thisHnd.Free();
+
+            // Allow context to be collected, which will cause resource cleanup in finalizer.
+            _ctx = null;
+        }
+        
+        private void Error(void* target, int errType, sbyte* errClsChars, int errClsCharsLen, sbyte* errMsgChars,
+            int errMsgCharsLen, void* errData, int errDataLen)
+        {
+            string errCls = IgniteUtils.Utf8UnmanagedToString(errClsChars, errClsCharsLen);
+            string errMsg = IgniteUtils.Utf8UnmanagedToString(errMsgChars, errMsgCharsLen);
+
+            switch (errType)
+            {
+                case ErrGeneric:
+                    if (_ignite != null && errDataLen > 0)
+                        throw ExceptionUtils.GetException(errCls, errMsg,
+                            _ignite.Marshaller.StartUnmarshal(new PlatformRawMemory(errData, errDataLen).Stream()));
+
+                    throw ExceptionUtils.GetException(errCls, errMsg);
+
+                case ErrJvmInit:
+                    throw ExceptionUtils.GetJvmInitializeException(errCls, errMsg);
+
+                case ErrJvmAttach:
+                    throw new IgniteException("Failed to attach to JVM.");
+
+                default:
+                    throw new IgniteException("Unknown exception [cls=" + errCls + ", msg=" + errMsg + ']');
+            }
+        }
+
+        #endregion
+        
+        #region HELPERS
+
+        private void SafeCall(Action func, bool allowUnitialized = false)
+        {
+            if (!allowUnitialized)
+                _initEvent.Wait();
+
+            try
+            {
+                func();
+            }
+            catch (Exception e)
+            {
+                UU.ThrowToJava(_ctx.NativeContext, e);
+            }
+        }
+
+        private T SafeCall<T>(Func<T> func, bool allowUnitialized = false)
+        {
+            if (!allowUnitialized)
+                _initEvent.Wait();
+
+            try
+            {
+                return func();
+            }
+            catch (Exception e)
+            {
+                UU.ThrowToJava(_ctx.NativeContext, e);
+
+                return default(T);
+            }
+        }
+
+        #endregion
+
+        /// <summary>
+        /// Callbacks pointer.
+        /// </summary>
+        public void* CallbacksPointer
+        {
+            get { return _cbsPtr.ToPointer(); }
+        }
+
+        /// <summary>
+        /// Gets the context.
+        /// </summary>
+        public UnmanagedContext Context
+        {
+            get { return _ctx; }
+        }
+
+        /// <summary>
+        /// Create function pointer for the given function.
+        /// </summary>
+        private void* CreateFunctionPointer(Delegate del)
+        {
+            _delegates.Add(del); // Prevent delegate from being GC-ed.
+
+            return Marshal.GetFunctionPointerForDelegate(del).ToPointer();
+        }
+
+        /// <param name="context">Context.</param>
+        public void SetContext(void* context)
+        {
+            Debug.Assert(context != null);
+            Debug.Assert(_ctx == null);
+
+            _ctx = new UnmanagedContext(context);
+        }
+
+        /// <summary>
+        /// Initializes this instance with grid.
+        /// </summary>
+        /// <param name="grid">Grid.</param>
+        public void Initialize(Ignite grid)
+        {
+            Debug.Assert(grid != null);
+
+            _ignite = grid;
+
+            lock (_initActions)
+            {
+                _initActions.ForEach(x => x(grid));
+
+                _initActions.Clear();
+            }
+
+            _initEvent.Set();
+        }
+
+        /// <summary>
+        /// Cleanups this instance.
+        /// </summary>
+        public void Cleanup()
+        {
+            _ignite = null;
+            
+            _handleRegistry.Close();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedContext.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedContext.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedContext.cs
new file mode 100644
index 0000000..89d2071
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedContext.cs
@@ -0,0 +1,53 @@
+/*
+ * 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.Unmanaged
+{
+    /// <summary>
+    /// Unmanaged context.
+    /// Wrapper around native ctx pointer to track finalization.
+    /// </summary>
+    internal unsafe class UnmanagedContext
+    {
+        /** Context */
+        private readonly void* _nativeCtx;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        public UnmanagedContext(void* ctx)
+        {
+            _nativeCtx = ctx;
+        }
+
+        /// <summary>
+        /// Gets the native context pointer.
+        /// </summary>
+        public void* NativeContext
+        {
+            get { return _nativeCtx; }
+        }
+
+        /// <summary>
+        /// Destructor.
+        /// </summary>
+        ~UnmanagedContext()
+        {
+            UnmanagedUtils.DeleteContext(_nativeCtx); // Release CPP object.
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedNonReleaseableTarget.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedNonReleaseableTarget.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedNonReleaseableTarget.cs
new file mode 100644
index 0000000..24db5a5
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedNonReleaseableTarget.cs
@@ -0,0 +1,68 @@
+/*
+ * 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.Unmanaged
+{
+    using System;
+
+    /// <summary>
+    /// Unmanaged target which does not require explicit release.
+    /// </summary>
+    internal unsafe class UnmanagedNonReleaseableTarget : IUnmanagedTarget
+    {
+        /** Context. */
+        private readonly void* _ctx;
+
+        /** Target. */
+        private readonly void* _target;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="ctx">Context.</param>
+        /// <param name="target">Target.</param>
+        public UnmanagedNonReleaseableTarget(void* ctx, void* target)
+        {
+            _ctx = ctx;
+            _target = target;
+        }
+
+        /** <inheritdoc /> */
+        public void* Context
+        {
+            get { return _ctx; }
+        }
+
+        /** <inheritdoc /> */
+        public void* Target
+        {
+            get { return _target; }
+        }
+
+        /** <inheritdoc /> */
+        public IUnmanagedTarget ChangeTarget(void* target)
+        {
+            throw new NotSupportedException();
+        }
+
+        /** <inheritdoc /> */
+        public void Dispose()
+        {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedTarget.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedTarget.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedTarget.cs
new file mode 100644
index 0000000..e54a199
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedTarget.cs
@@ -0,0 +1,77 @@
+/*
+ * 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.Unmanaged
+{
+    using System;
+    using System.Runtime.InteropServices;
+    using UU = UnmanagedUtils;
+
+    /// <summary>
+    /// Base unmanaged target implementation.
+    /// </summary>
+    internal unsafe sealed class UnmanagedTarget : CriticalHandle, IUnmanagedTarget
+    {
+        /** Context. */
+        private readonly UnmanagedContext _ctx;
+        
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="ctx">Context.</param>
+        /// <param name="target">Target.</param>
+        public UnmanagedTarget(UnmanagedContext ctx, void* target)
+            : base(IntPtr.Zero)
+        {
+            _ctx = ctx;
+            
+            SetHandle(new IntPtr(target));
+        }
+
+        /** <inheritdoc /> */
+        public void* Context
+        {
+            get { return _ctx.NativeContext; }
+        }
+
+        /** <inheritdoc /> */
+        public void* Target
+        {
+            get { return handle.ToPointer(); }
+        }
+
+        /** <inheritdoc /> */
+        public IUnmanagedTarget ChangeTarget(void* target)
+        {
+            return new UnmanagedTarget(_ctx, target);
+        }
+
+        /** <inheritdoc /> */
+        protected override bool ReleaseHandle()
+        {
+            UU.Release(this);
+            
+            return true;
+        }
+
+        /** <inheritdoc /> */
+        public override bool IsInvalid
+        {
+            get { return handle == IntPtr.Zero; }
+        }
+    }
+}


[43/45] ignite git commit: 1.5.0-SNAPSHOT

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


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

Branch: refs/heads/ignite-843
Commit: 2b16b57438cd03a60e561da10df869adf893418a
Parents: de4a26e
Author: Ignite Teamcity <ig...@apache.org>
Authored: Fri Sep 4 18:14:23 2015 +0300
Committer: Ignite Teamcity <ig...@apache.org>
Committed: Fri Sep 4 18:14:23 2015 +0300

----------------------------------------------------------------------
 .../platform/src/main/cpp/common/configure.ac   |  2 +-
 .../src/main/cpp/core-test/configure.ac         |  2 +-
 modules/platform/src/main/cpp/core/configure.ac |  2 +-
 .../Properties/AssemblyInfo.cs                  |  4 +-
 .../Properties/AssemblyInfo.cs                  |  4 +-
 .../Properties/AssemblyInfo.cs                  |  4 +-
 pom.xml                                         | 45 ++++++++++----------
 7 files changed, 31 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/2b16b574/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b506889..6f9c848 100644
--- a/pom.xml
+++ b/pom.xml
@@ -780,41 +780,40 @@
                                             setClientVersion('new.ignite.version.fixed', 'new.client.version');
                                         </script>
 
-                                        <echo message="Update ignite.version in dotnet client"/>
-                                        <echo message="${new.client.version}"/>
+                                        <echo message="Update ignite.version in dotnet client" />
+                                        <echo message="${new.client.version}" />
 
                                         <replaceregexp byline="true" encoding="UTF-8">
-                                            <regexp
-                                                pattern="(\[assembly:\s*Assembly\w*Version\w*\(&quot;)\d.\d.\d(.\d)?(&quot;\)\])"/>
-                                            <substitution expression="\1${new.client.version}\3"/>
+                                            <regexp pattern="(\[assembly:\s*Assembly\w*Version\w*\(&quot;)\d.\d.\d(.\d)?(&quot;\)\])" />
+                                            <substitution expression="\1${new.client.version}\3" />
                                             <fileset dir="${basedir}/">
-                                                <include name="**/AssemblyInfo.cs"/>
-                                                <include name="**/AssemblyInfo.cpp"/>
+                                                <include name="**/AssemblyInfo.cs" />
+                                                <include name="**/AssemblyInfo.cpp" />
                                             </fileset>
                                         </replaceregexp>
 
-                                        <echo message="Update ignite.version in cpp client"/>
+                                        <echo message="Update ignite.version in cpp client" />
                                         <replaceregexp byline="true" encoding="UTF-8">
-                                            <regexp pattern="(AC_INIT.+\[)\d.\d.\d(.\d)?(\].+)"/>
-                                            <substitution expression="\1${new.client.version}\3"/>
+                                            <regexp pattern="(AC_INIT.+\[)\d.\d.\d(.\d)?(\].+)" />
+                                            <substitution expression="\1${new.client.version}\3" />
                                             <fileset dir="${basedir}/">
-                                                <include name="**/configure.ac"/>
+                                                <include name="**/configure.ac" />
                                             </fileset>
                                         </replaceregexp>
 
                                         <replaceregexp byline="true" encoding="UTF-8">
-                                            <regexp pattern="(define GG_VERSION_STR_WIN &quot;)\d.\d.\d(.\d)?(&quot;)"/>
-                                            <substitution expression="\1${new.client.version}\3"/>
+                                            <regexp pattern="(define GG_VERSION_STR_WIN &quot;)\d.\d.\d(.\d)?(&quot;)" />
+                                            <substitution expression="\1${new.client.version}\3" />
                                             <fileset dir="${basedir}/">
-                                                <include name="**/resource.h"/>
+                                                <include name="**/resource.h" />
                                             </fileset>
                                         </replaceregexp>
 
                                         <replaceregexp byline="true" encoding="UTF-16">
-                                            <regexp pattern="(Version&quot;, &quot;)\d.\d.\d.\d(&quot;)"/>
-                                            <substitution expression="\1${new.client.version}\2"/>
+                                            <regexp pattern="(Version&quot;, &quot;)\d.\d.\d.\d(&quot;)" />
+                                            <substitution expression="\1${new.client.version}\2" />
                                             <fileset dir="${basedir}/">
-                                                <include name="**/Resource.rc"/>
+                                                <include name="**/Resource.rc" />
                                             </fileset>
                                         </replaceregexp>
 
@@ -833,18 +832,18 @@
                                         </script>
 
                                         <replaceregexp byline="true" encoding="UTF-8">
-                                            <regexp pattern="(define GG_VERSION_BIN_WIN )\d,\d,\d,\d"/>
-                                            <substitution expression="\1${new.bin.version}"/>
+                                            <regexp pattern="(define GG_VERSION_BIN_WIN )\d,\d,\d,\d" />
+                                            <substitution expression="\1${new.bin.version}" />
                                             <fileset dir="${basedir}/">
-                                                <include name="**/resource.h"/>
+                                                <include name="**/resource.h" />
                                             </fileset>
                                         </replaceregexp>
 
                                         <replaceregexp byline="true" encoding="UTF-16">
-                                            <regexp pattern="(VERSION )\d,\d,\d,\d"/>
-                                            <substitution expression="\1${new.bin.version}"/>
+                                            <regexp pattern="(VERSION )\d,\d,\d,\d" />
+                                            <substitution expression="\1${new.bin.version}" />
                                             <fileset dir="${basedir}/">
-                                                <include name="**/Resource.rc"/>
+                                                <include name="**/Resource.rc" />
                                             </fileset>
                                         </replaceregexp>
                                     </target>


[31/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/AbstractQueryCursor.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/AbstractQueryCursor.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/AbstractQueryCursor.cs
new file mode 100644
index 0000000..0f4b5a3
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/AbstractQueryCursor.cs
@@ -0,0 +1,264 @@
+/*
+ * 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.Cache.Query
+{
+    using System;
+    using System.Collections;
+    using System.Collections.Generic;
+    using System.Diagnostics.CodeAnalysis;
+    using Apache.Ignite.Core.Cache.Query;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+    using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils;
+
+    /// <summary>
+    /// Abstract query cursor implementation.
+    /// </summary>
+    internal abstract class AbstractQueryCursor<T> : PlatformDisposableTarget, IQueryCursor<T>, IEnumerator<T>
+    {
+        /** */
+        private const int OpGetAll = 1;
+
+        /** */
+        private const int OpGetBatch = 2;
+
+        /** Position before head. */
+        private const int BatchPosBeforeHead = -1;
+
+        /** Keep portable flag. */
+        private readonly bool _keepPortable;
+
+        /** Wherther "GetAll" was called. */
+        private bool _getAllCalled;
+
+        /** Whether "GetEnumerator" was called. */
+        private bool _iterCalled;
+
+        /** Batch with entries. */
+        private T[] _batch;
+
+        /** Current position in batch. */
+        private int _batchPos = BatchPosBeforeHead;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="target">Target.</param>
+        /// <param name="marsh">Marshaller.</param>
+        /// <param name="keepPortable">Keep portable flag.</param>
+        protected AbstractQueryCursor(IUnmanagedTarget target, PortableMarshaller marsh, bool keepPortable) : 
+            base(target, marsh)
+        {
+            _keepPortable = keepPortable;
+        }
+
+        #region Public methods
+
+        /** <inheritdoc /> */
+        public IList<T> GetAll()
+        {
+            ThrowIfDisposed();
+
+            if (_iterCalled)
+                throw new InvalidOperationException("Failed to get all entries because GetEnumerator() " + 
+                    "method has already been called.");
+
+            if (_getAllCalled)
+                throw new InvalidOperationException("Failed to get all entries because GetAll() " + 
+                    "method has already been called.");
+
+            var res = DoInOp<IList<T>>(OpGetAll, ConvertGetAll);
+
+            _getAllCalled = true;
+
+            return res;
+        }
+
+        /** <inheritdoc /> */
+        protected override void Dispose(bool disposing)
+        {
+            try
+            {
+                UU.QueryCursorClose(Target);
+            }
+            finally 
+            {
+                base.Dispose(disposing);
+            }
+        }
+
+        #endregion
+
+        #region Public IEnumerable methods
+
+        /** <inheritdoc /> */
+        [SuppressMessage("ReSharper", "PossibleNullReferenceException")]
+        public IEnumerator<T> GetEnumerator()
+        {
+            ThrowIfDisposed();
+
+            if (_iterCalled)
+                throw new InvalidOperationException("Failed to get enumerator entries because " + 
+                    "GetEnumeartor() method has already been called.");
+
+            if (_getAllCalled)
+                throw new InvalidOperationException("Failed to get enumerator entries because " + 
+                    "GetAll() method has already been called.");
+
+            UU.QueryCursorIterator(Target);
+
+            _iterCalled = true;
+
+            return this;
+        }
+
+        /** <inheritdoc /> */
+        IEnumerator IEnumerable.GetEnumerator()
+        {
+            return GetEnumerator();
+        }
+
+        #endregion
+
+        #region Public IEnumerator methods
+
+        /** <inheritdoc /> */
+        public T Current
+        {
+            get
+            {
+                ThrowIfDisposed();
+
+                if (_batchPos == BatchPosBeforeHead)
+                    throw new InvalidOperationException("MoveNext has not been called.");
+                
+                if (_batch == null)
+                    throw new InvalidOperationException("Previous call to MoveNext returned false.");
+
+                return _batch[_batchPos];
+            }
+        }
+
+        /** <inheritdoc /> */
+        object IEnumerator.Current
+        {
+            get { return Current; }
+        }
+
+        /** <inheritdoc /> */
+        public bool MoveNext()
+        {
+            ThrowIfDisposed();
+
+            if (_batch == null)
+            {
+                if (_batchPos == BatchPosBeforeHead)
+                    // Standing before head, let's get batch and advance position.
+                    RequestBatch();
+            }
+            else
+            {
+                _batchPos++;
+
+                if (_batch.Length == _batchPos)
+                    // Reached batch end => request another.
+                    RequestBatch();
+            }
+
+            return _batch != null;
+        }
+
+        /** <inheritdoc /> */
+        public void Reset()
+        {
+            throw new NotSupportedException("Reset is not supported.");
+        }
+
+        #endregion
+
+        #region Non-public methods
+
+        /// <summary>
+        /// Read entry from the reader.
+        /// </summary>
+        /// <param name="reader">Reader.</param>
+        /// <returns>Entry.</returns>
+        protected abstract T Read(PortableReaderImpl reader);
+
+        /** <inheritdoc /> */
+        protected override T1 Unmarshal<T1>(IPortableStream stream)
+        {
+            return Marshaller.Unmarshal<T1>(stream, _keepPortable);
+        }
+
+        /// <summary>
+        /// Request next batch.
+        /// </summary>
+        private void RequestBatch()
+        {
+            _batch = DoInOp<T[]>(OpGetBatch, ConvertGetBatch);
+
+            _batchPos = 0;
+        }
+
+        /// <summary>
+        /// Converter for GET_ALL operation.
+        /// </summary>
+        /// <param name="stream">Portable stream.</param>
+        /// <returns>Result.</returns>
+        private IList<T> ConvertGetAll(IPortableStream stream)
+        {
+            var reader = Marshaller.StartUnmarshal(stream, _keepPortable);
+
+            var size = reader.ReadInt();
+
+            var res = new List<T>(size);
+
+            for (var i = 0; i < size; i++)
+                res.Add(Read(reader));
+
+            return res;
+        }
+
+        /// <summary>
+        /// Converter for GET_BATCH operation.
+        /// </summary>
+        /// <param name="stream">Portable stream.</param>
+        /// <returns>Result.</returns>
+        private T[] ConvertGetBatch(IPortableStream stream)
+        {
+            var reader = Marshaller.StartUnmarshal(stream, _keepPortable);
+
+            var size = reader.ReadInt();
+
+            if (size == 0)
+                return null;
+
+            var res = new T[size];
+
+            for (var i = 0; i < size; i++)
+                res[i] = Read(reader);
+
+            return res;
+        }
+
+        #endregion
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryFilter.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryFilter.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryFilter.cs
new file mode 100644
index 0000000..5738ed9
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryFilter.cs
@@ -0,0 +1,125 @@
+/*
+ * 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.Cache.Query.Continuous
+{
+    using Apache.Ignite.Core.Cache.Event;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Impl.Resource;
+    using CQU = ContinuousQueryUtils;
+
+    /// <summary>
+    /// Continuous query filter interface. Required to hide generic nature of underliyng real filter.
+    /// </summary>
+    internal interface IContinuousQueryFilter
+    {
+        /// <summary>
+        /// Evaluate filter.
+        /// </summary>
+        /// <param name="stream">Stream.</param>
+        /// <returns>Result.</returns>
+        bool Evaluate(IPortableStream stream);
+
+        /// <summary>
+        /// Inject grid.
+        /// </summary>
+        /// <param name="grid"></param>
+        void Inject(Ignite grid);
+
+        /// <summary>
+        /// Allocate handle for the filter.
+        /// </summary>
+        /// <returns></returns>
+        long Allocate();
+
+        /// <summary>
+        /// Release filter.
+        /// </summary>
+        void Release();
+    }
+
+    /// <summary>
+    /// Continuous query filter generic implementation.
+    /// </summary>
+    internal class ContinuousQueryFilter<TK, TV> : IContinuousQueryFilter        
+    {
+        /** Actual filter. */
+        private readonly ICacheEntryEventFilter<TK, TV> _filter;
+
+        /** Keep portable flag. */
+        private readonly bool _keepPortable;
+
+        /** Ignite hosting the filter. */
+        private volatile Ignite _ignite;
+
+        /** GC handle. */
+        private long? _hnd;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="filter">Actual filter.</param>
+        /// <param name="keepPortable">Keep portable flag.</param>
+        public ContinuousQueryFilter(ICacheEntryEventFilter<TK, TV> filter, bool keepPortable)
+        {
+            _filter = filter;
+            _keepPortable = keepPortable;
+        }
+
+        /** <inheritDoc /> */
+        public bool Evaluate(IPortableStream stream)
+        {
+            ICacheEntryEvent<TK, TV> evt = CQU.ReadEvent<TK, TV>(stream, _ignite.Marshaller, _keepPortable);
+
+            return _filter.Evaluate(evt);
+        }
+
+        /** <inheritDoc /> */
+        public void Inject(Ignite grid)
+        {
+            _ignite = grid;
+
+            ResourceProcessor.Inject(_filter, grid);
+        }
+
+        /** <inheritDoc /> */
+        public long Allocate()
+        {
+            lock (this)
+            {
+                if (!_hnd.HasValue)
+                    _hnd = _ignite.HandleRegistry.Allocate(this);
+
+                return _hnd.Value;
+            }
+        }
+
+        /** <inheritDoc /> */
+        public void Release()
+        {
+            lock (this)
+            {
+                if (_hnd.HasValue)
+                {
+                    _ignite.HandleRegistry.Release(_hnd.Value);
+
+                    _hnd = null;
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryFilterHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryFilterHolder.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryFilterHolder.cs
new file mode 100644
index 0000000..65da674
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryFilterHolder.cs
@@ -0,0 +1,118 @@
+/*
+ * 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.Cache.Query.Continuous
+{
+    using System;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Continuous query remote filter holder. Wraps real filter into portable object,
+    /// so that it can be passed over wire to another node.
+    /// </summary>
+    public class ContinuousQueryFilterHolder : IPortableWriteAware
+    {
+        /** Key type. */
+        private readonly Type _keyTyp;
+
+        /** Value type. */
+        private readonly Type _valTyp;
+
+        /** Filter object. */
+        private readonly object _filter;
+
+        /** Keep portable flag. */
+        private readonly bool _keepPortable;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="keyTyp">Key type.</param>
+        /// <param name="valTyp">Value type.</param>
+        /// <param name="filter">Filter.</param>
+        /// <param name="keepPortable">Keep portable flag.</param>
+        public ContinuousQueryFilterHolder(Type keyTyp, Type valTyp, object filter, bool keepPortable)
+        {
+            _keyTyp = keyTyp;
+            _valTyp = valTyp;
+            _filter = filter;
+            _keepPortable = keepPortable;
+        }
+
+        /// <summary>
+        /// Key type.
+        /// </summary>
+        internal Type KeyType
+        {
+            get { return _keyTyp; }
+        }
+
+        /// <summary>
+        /// Value type.
+        /// </summary>
+        internal Type ValueType
+        {
+            get { return _valTyp; }
+        }
+
+        /// <summary>
+        /// Filter.
+        /// </summary>
+        internal object Filter
+        {
+            get { return _filter; }
+        }
+
+        /// <summary>
+        /// Keep portable flag.
+        /// </summary>
+        internal bool KeepPortable
+        {
+            get { return _keepPortable; }
+        }
+
+        /// <summary>
+        /// Writes this object to the given writer.
+        /// </summary>
+        /// <param name="writer">Writer.</param>
+        public void WritePortable(IPortableWriter writer)
+        {
+            PortableWriterImpl rawWriter = (PortableWriterImpl) writer.RawWriter();
+
+            PortableUtils.WritePortableOrSerializable(rawWriter, _keyTyp);
+            PortableUtils.WritePortableOrSerializable(rawWriter, _valTyp);
+            PortableUtils.WritePortableOrSerializable(rawWriter, _filter);
+
+            rawWriter.WriteBoolean(_keepPortable);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ContinuousQueryFilterHolder"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        public ContinuousQueryFilterHolder(IPortableReader reader)
+        {
+            PortableReaderImpl rawReader = (PortableReaderImpl) reader.RawReader();
+
+            _keyTyp = PortableUtils.ReadPortableOrSerializable<Type>(rawReader);
+            _valTyp = PortableUtils.ReadPortableOrSerializable<Type>(rawReader);
+            _filter = PortableUtils.ReadPortableOrSerializable<object>(rawReader);
+            _keepPortable = rawReader.ReadBoolean();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryHandleImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryHandleImpl.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryHandleImpl.cs
new file mode 100644
index 0000000..7a1b544
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryHandleImpl.cs
@@ -0,0 +1,216 @@
+/*
+ * 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.Cache.Query.Continuous
+{
+    using System;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cache.Event;
+    using Apache.Ignite.Core.Cache.Query;
+    using Apache.Ignite.Core.Cache.Query.Continuous;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Impl.Resource;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+    using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils;
+    using CQU = ContinuousQueryUtils;
+
+    /// <summary>
+    /// Continuous query handle interface.
+    /// </summary>
+    internal interface IContinuousQueryHandleImpl : IDisposable
+    {
+        /// <summary>
+        /// Process callback.
+        /// </summary>
+        /// <param name="stream">Stream.</param>
+        /// <returns>Result.</returns>
+        void Apply(IPortableStream stream);
+    }
+
+    /// <summary>
+    /// Continuous query handle.
+    /// </summary>
+    internal class ContinuousQueryHandleImpl<TK, TV> : IContinuousQueryHandleImpl, IContinuousQueryFilter, 
+        IContinuousQueryHandle<ICacheEntry<TK, TV>>
+    {
+        /** Marshaller. */
+        private readonly PortableMarshaller _marsh;
+
+        /** Keep portable flag. */
+        private readonly bool _keepPortable;
+
+        /** Real listener. */
+        private readonly ICacheEntryEventListener<TK, TV> _lsnr;
+
+        /** Real filter. */
+        private readonly ICacheEntryEventFilter<TK, TV> _filter;
+
+        /** GC handle. */
+        private long _hnd;
+
+        /** Native query. */
+        private volatile IUnmanagedTarget _nativeQry;
+        
+        /** Initial query cursor. */
+        private volatile IQueryCursor<ICacheEntry<TK, TV>> _initialQueryCursor;
+
+        /** Disposed flag. */
+        private bool _disposed;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="qry">Query.</param>
+        /// <param name="marsh">Marshaller.</param>
+        /// <param name="keepPortable">Keep portable flag.</param>
+        public ContinuousQueryHandleImpl(ContinuousQuery<TK, TV> qry, PortableMarshaller marsh, bool keepPortable)
+        {
+            _marsh = marsh;
+            _keepPortable = keepPortable;
+
+            _lsnr = qry.Listener;
+            _filter = qry.Filter;
+        }
+
+        /// <summary>
+        /// Start execution.
+        /// </summary>
+        /// <param name="grid">Ignite instance.</param>
+        /// <param name="writer">Writer.</param>
+        /// <param name="cb">Callback invoked when all necessary data is written to stream.</param>
+        /// <param name="qry">Query.</param>
+        public void Start(Ignite grid, PortableWriterImpl writer, Func<IUnmanagedTarget> cb, 
+            ContinuousQuery<TK, TV> qry)
+        {
+            // 1. Inject resources.
+            ResourceProcessor.Inject(_lsnr, grid);
+            ResourceProcessor.Inject(_filter, grid);
+
+            // 2. Allocate handle.
+            _hnd = grid.HandleRegistry.Allocate(this);
+
+            // 3. Write data to stream.
+            writer.WriteLong(_hnd);
+            writer.WriteBoolean(qry.Local);
+            writer.WriteBoolean(_filter != null);
+
+            ContinuousQueryFilterHolder filterHolder = _filter == null || qry.Local ? null : 
+                new ContinuousQueryFilterHolder(typeof (TK), typeof (TV), _filter, _keepPortable);
+
+            writer.WriteObject(filterHolder);
+
+            writer.WriteInt(qry.BufferSize);
+            writer.WriteLong((long)qry.TimeInterval.TotalMilliseconds);
+            writer.WriteBoolean(qry.AutoUnsubscribe);
+
+            // 4. Call Java.
+            _nativeQry = cb();
+
+            // 5. Initial query.
+            var nativeInitialQryCur = UU.ContinuousQueryGetInitialQueryCursor(_nativeQry);
+            _initialQueryCursor = nativeInitialQryCur == null
+                ? null
+                : new QueryCursor<TK, TV>(nativeInitialQryCur, _marsh, _keepPortable);
+        }
+
+        /** <inheritdoc /> */
+        public void Apply(IPortableStream stream)
+        {
+            ICacheEntryEvent<TK, TV>[] evts = CQU.ReadEvents<TK, TV>(stream, _marsh, _keepPortable);
+
+            _lsnr.OnEvent(evts); 
+        }
+
+        /** <inheritdoc /> */
+        public bool Evaluate(IPortableStream stream)
+        {
+            Debug.Assert(_filter != null, "Evaluate should not be called if filter is not set.");
+
+            ICacheEntryEvent<TK, TV> evt = CQU.ReadEvent<TK, TV>(stream, _marsh, _keepPortable);
+
+            return _filter.Evaluate(evt);
+        }
+
+        /** <inheritdoc /> */
+        public void Inject(Ignite grid)
+        {
+            throw new NotSupportedException("Should not be called.");
+        }
+
+        /** <inheritdoc /> */
+        public long Allocate()
+        {
+            throw new NotSupportedException("Should not be called.");
+        }
+
+        /** <inheritdoc /> */
+        public void Release()
+        {
+            _marsh.Ignite.HandleRegistry.Release(_hnd);
+        }
+
+        /** <inheritdoc /> */
+        public IQueryCursor<ICacheEntry<TK, TV>> InitialQueryCursor
+        {
+            get { return GetInitialQueryCursor(); }
+        }
+
+        /** <inheritdoc /> */
+        public IQueryCursor<ICacheEntry<TK, TV>> GetInitialQueryCursor()
+        {
+            lock (this)
+            {
+                if (_disposed)
+                    throw new ObjectDisposedException("Continuous query handle has been disposed.");
+
+                var cur = _initialQueryCursor;
+
+                if (cur == null)
+                    throw new InvalidOperationException("GetInitialQueryCursor() can be called only once.");
+
+                _initialQueryCursor = null;
+
+                return cur;
+            }
+        }
+
+        /** <inheritdoc /> */
+        public void Dispose()
+        {
+            lock (this)
+            {
+                if (_disposed)
+                    return;
+
+                Debug.Assert(_nativeQry != null);
+
+                try
+                {
+                    UU.ContinuousQueryClose(_nativeQry);
+                }
+                finally
+                {
+                    _nativeQry.Dispose();
+
+                    _disposed = true;
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryUtils.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryUtils.cs
new file mode 100644
index 0000000..86c8300
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryUtils.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.Cache.Query.Continuous
+{
+    using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
+    using Apache.Ignite.Core.Cache.Event;
+    using Apache.Ignite.Core.Impl.Cache.Event;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+
+    /// <summary>
+    /// Utility methods for continuous queries.
+    /// </summary>
+    static class ContinuousQueryUtils
+    {
+        /// <summary>
+        /// Read single event.
+        /// </summary>
+        /// <param name="stream">Stream to read data from.</param>
+        /// <param name="marsh">Marshaller.</param>
+        /// <param name="keepPortable">Keep portable flag.</param>
+        /// <returns>Event.</returns>
+        public static ICacheEntryEvent<TK, TV> ReadEvent<TK, TV>(IPortableStream stream, 
+            PortableMarshaller marsh, bool keepPortable)
+        {
+            var reader = marsh.StartUnmarshal(stream, keepPortable);
+
+            return ReadEvent0<TK, TV>(reader);
+        }
+
+        /// <summary>
+        /// Read multiple events.
+        /// </summary>
+        /// <param name="stream">Stream.</param>
+        /// <param name="marsh">Marshaller.</param>
+        /// <param name="keepPortable">Keep portable flag.</param>
+        /// <returns>Events.</returns>
+        [SuppressMessage("ReSharper", "PossibleNullReferenceException")]
+        public static ICacheEntryEvent<TK, TV>[] ReadEvents<TK, TV>(IPortableStream stream,
+            PortableMarshaller marsh, bool keepPortable)
+        {
+            var reader = marsh.StartUnmarshal(stream, keepPortable);
+
+            int cnt = reader.ReadInt();
+
+            ICacheEntryEvent<TK, TV>[] evts = new ICacheEntryEvent<TK, TV>[cnt];
+
+            for (int i = 0; i < cnt; i++)
+                evts[i] = ReadEvent0<TK, TV>(reader);
+
+            return evts;
+        }
+
+        /// <summary>
+        /// Read event.
+        /// </summary>
+        /// <param name="reader">Reader.</param>
+        /// <returns>Event.</returns>
+        private static ICacheEntryEvent<TK, TV> ReadEvent0<TK, TV>(PortableReaderImpl reader)
+        {
+            reader.DetachNext();
+            TK key = reader.ReadObject<TK>();
+
+            reader.DetachNext();
+            TV oldVal = reader.ReadObject<TV>();
+
+            reader.DetachNext();
+            TV val = reader.ReadObject<TV>();
+
+            return CreateEvent(key, oldVal, val);
+        }
+
+        /// <summary>
+        /// Create event.
+        /// </summary>
+        /// <param name="key">Key.</param>
+        /// <param name="oldVal">Old value.</param>
+        /// <param name="val">Value.</param>
+        /// <returns>Event.</returns>
+        public static ICacheEntryEvent<TK, TV> CreateEvent<TK, TV>(TK key, TV oldVal, TV val)
+        {
+            if (oldVal == null)
+            {
+                Debug.Assert(val != null);
+
+                return new CacheEntryCreateEvent<TK, TV>(key, val);
+            }
+
+            if (val == null)
+            {
+                Debug.Assert(oldVal != null);
+
+                return new CacheEntryRemoveEvent<TK, TV>(key, oldVal);
+            }
+            
+            return new CacheEntryUpdateEvent<TK, TV>(key, oldVal, val);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/FieldsQueryCursor.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/FieldsQueryCursor.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/FieldsQueryCursor.cs
new file mode 100644
index 0000000..f38346c
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/FieldsQueryCursor.cs
@@ -0,0 +1,54 @@
+/*
+ * 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.Cache.Query
+{
+    using System.Collections;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+
+    /// <summary>
+    /// Cursor for entry-based queries.
+    /// </summary>
+    internal class FieldsQueryCursor : AbstractQueryCursor<IList>
+    {
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="target">Target.</param>
+        /// <param name="marsh">Marshaler.</param>
+        /// <param name="keepPortable">Keep poratble flag.</param>
+        public FieldsQueryCursor(IUnmanagedTarget target, PortableMarshaller marsh, bool keepPortable)
+            : base(target, marsh, keepPortable)
+        {
+            // No-op.
+        }
+
+        /** <inheritdoc /> */
+        protected override IList Read(PortableReaderImpl reader)
+        {
+            int cnt = reader.ReadInt();
+
+            var res = new ArrayList(cnt);
+
+            for (int i = 0; i < cnt; i++)
+                res.Add(reader.ReadObject<object>());
+
+            return res;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/QueryCursor.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/QueryCursor.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/QueryCursor.cs
new file mode 100644
index 0000000..0b113f5
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Query/QueryCursor.cs
@@ -0,0 +1,50 @@
+/*
+ * 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.Cache.Query
+{
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+
+    /// <summary>
+    /// Cursor for entry-based queries.
+    /// </summary>
+    internal class QueryCursor<TK, TV> : AbstractQueryCursor<ICacheEntry<TK, TV>>
+    {
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="target">Target.</param>
+        /// <param name="marsh">Marshaler.</param>
+        /// <param name="keepPortable">Keep poratble flag.</param>
+        public QueryCursor(IUnmanagedTarget target, PortableMarshaller marsh,
+            bool keepPortable) : base(target, marsh, keepPortable)
+        {
+            // No-op.
+        }
+
+        /** <inheritdoc /> */
+        protected override ICacheEntry<TK, TV> Read(PortableReaderImpl reader)
+        {
+            TK key = reader.ReadObject<TK>();
+            TV val = reader.ReadObject<TV>();
+
+            return new CacheEntry<TK, TV>(key, val);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStore.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStore.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStore.cs
new file mode 100644
index 0000000..3fbc705
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStore.cs
@@ -0,0 +1,263 @@
+/*
+ * 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.Cache.Store
+{
+    using System.Collections;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Cache.Store;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Handle;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+    using Apache.Ignite.Core.Impl.Resource;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Interop cache store.
+    /// </summary>
+    internal class CacheStore
+    {
+        /** */
+        private const byte OpLoadCache = 0;
+
+        /** */
+        private const byte OpLoad = 1;
+
+        /** */
+        private const byte OpLoadAll = 2;
+
+        /** */
+        private const byte OpPut = 3;
+
+        /** */
+        private const byte OpPutAll = 4;
+
+        /** */
+        private const byte OpRmv = 5;
+
+        /** */
+        private const byte OpRmvAll = 6;
+
+        /** */
+        private const byte OpSesEnd = 7;
+        
+        /** */
+        private readonly bool _convertPortable;
+
+        /** Store. */
+        private readonly ICacheStore _store;
+
+        /** Session. */
+        private readonly CacheStoreSessionProxy _sesProxy;
+
+        /** */
+        private readonly long _handle;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheStore" /> class.
+        /// </summary>
+        /// <param name="store">Store.</param>
+        /// <param name="convertPortable">Whether to convert portable objects.</param>
+        /// <param name="registry">The handle registry.</param>
+        private CacheStore(ICacheStore store, bool convertPortable, HandleRegistry registry)
+        {
+            Debug.Assert(store != null);
+
+            _store = store;
+            _convertPortable = convertPortable;
+
+            _sesProxy = new CacheStoreSessionProxy();
+
+            ResourceProcessor.InjectStoreSession(store, _sesProxy);
+
+            _handle = registry.AllocateCritical(this);
+        }
+
+        /// <summary>
+        /// Creates interop cache store from a stream.
+        /// </summary>
+        /// <param name="memPtr">Memory pointer.</param>
+        /// <param name="registry">The handle registry.</param>
+        /// <returns>
+        /// Interop cache store.
+        /// </returns>
+        internal static CacheStore CreateInstance(long memPtr, HandleRegistry registry)
+        {
+            using (var stream = IgniteManager.Memory.Get(memPtr).Stream())
+            {
+                var reader = PortableUtils.Marshaller.StartUnmarshal(stream, PortableMode.KeepPortable);
+
+                var assemblyName = reader.ReadString();
+                var className = reader.ReadString();
+                var convertPortable = reader.ReadBoolean();
+                var propertyMap = reader.ReadGenericDictionary<string, object>();
+
+                var store = (ICacheStore) IgniteUtils.CreateInstance(assemblyName, className);
+
+                IgniteUtils.SetProperties(store, propertyMap);
+
+                return new CacheStore(store, convertPortable, registry);
+            }
+        }
+
+        /// <summary>
+        /// Gets the handle.
+        /// </summary>
+        public long Handle
+        {
+            get { return _handle; }
+        }
+
+        /// <summary>
+        /// Initializes this instance with a grid.
+        /// </summary>
+        /// <param name="grid">Grid.</param>
+        public void Init(Ignite grid)
+        {
+            ResourceProcessor.Inject(_store, grid);
+        }
+
+        /// <summary>
+        /// Invokes a store operation.
+        /// </summary>
+        /// <param name="input">Input stream.</param>
+        /// <param name="cb">Callback.</param>
+        /// <param name="grid">Grid.</param>
+        /// <returns>Invocation result.</returns>
+        /// <exception cref="IgniteException">Invalid operation type:  + opType</exception>
+        public int Invoke(IPortableStream input, IUnmanagedTarget cb, Ignite grid)
+        {
+            IPortableReader reader = grid.Marshaller.StartUnmarshal(input,
+                _convertPortable ? PortableMode.Deserialize : PortableMode.ForcePortable);
+            
+            IPortableRawReader rawReader = reader.RawReader();
+
+            int opType = rawReader.ReadByte();
+
+            // Setup cache sessoin for this invocation.
+            long sesId = rawReader.ReadLong();
+            
+            CacheStoreSession ses = grid.HandleRegistry.Get<CacheStoreSession>(sesId, true);
+
+            ses.CacheName = rawReader.ReadString();
+
+            _sesProxy.SetSession(ses);
+
+            try
+            {
+                // Perform operation.
+                switch (opType)
+                {
+                    case OpLoadCache:
+                        _store.LoadCache((k, v) => WriteObjects(cb, grid, k, v), rawReader.ReadObjectArray<object>());
+
+                        break;
+
+                    case OpLoad:
+                        object val = _store.Load(rawReader.ReadObject<object>());
+
+                        if (val != null)
+                            WriteObjects(cb, grid, val);
+
+                        break;
+
+                    case OpLoadAll:
+                        var keys = rawReader.ReadCollection();
+
+                        var result = _store.LoadAll(keys);
+
+                        foreach (DictionaryEntry entry in result)
+                            WriteObjects(cb, grid, entry.Key, entry.Value);
+
+                        break;
+
+                    case OpPut:
+                        _store.Write(rawReader.ReadObject<object>(), rawReader.ReadObject<object>());
+
+                        break;
+
+                    case OpPutAll:
+                        _store.WriteAll(rawReader.ReadDictionary());
+
+                        break;
+
+                    case OpRmv:
+                        _store.Delete(rawReader.ReadObject<object>());
+
+                        break;
+
+                    case OpRmvAll:
+                        _store.DeleteAll(rawReader.ReadCollection());
+
+                        break;
+
+                    case OpSesEnd:
+                        grid.HandleRegistry.Release(sesId);
+
+                        _store.SessionEnd(rawReader.ReadBoolean());
+
+                        break;
+
+                    default:
+                        throw new IgniteException("Invalid operation type: " + opType);
+                }
+
+                return 0;
+            }
+            finally
+            {
+                _sesProxy.ClearSession();
+            }
+        }
+
+        /// <summary>
+        /// Writes objects to the marshaller.
+        /// </summary>
+        /// <param name="cb">Optional callback.</param>
+        /// <param name="grid">Grid.</param>
+        /// <param name="objects">Objects.</param>
+        private static void WriteObjects(IUnmanagedTarget cb, Ignite grid, params object[] objects)
+        {
+            using (var stream = IgniteManager.Memory.Allocate().Stream())
+            {
+                PortableWriterImpl writer = grid.Marshaller.StartMarshal(stream);
+
+                try
+                {
+                    foreach (var obj in objects)
+                    {
+                        writer.DetachNext();
+                        writer.WriteObject(obj);
+                    }
+                }
+                finally
+                {
+                    grid.Marshaller.FinishMarshal(writer);
+                }
+
+                if (cb != null)
+                {
+                    stream.SynchronizeOutput();
+
+                    UnmanagedUtils.CacheStoreCallbackInvoke(cb, stream.MemoryPointer);
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStoreSession.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStoreSession.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStoreSession.cs
new file mode 100644
index 0000000..f771fe8
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStoreSession.cs
@@ -0,0 +1,53 @@
+/*
+ * 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.Cache.Store
+{
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Cache.Store;
+
+    /// <summary>
+    /// Store session implementation.
+    /// </summary>
+    internal class CacheStoreSession : ICacheStoreSession
+    {
+        /** Properties. */
+        private IDictionary<object, object> _props;
+        
+        /** <inheritdoc /> */
+
+        public string CacheName
+        {
+            get; internal set;
+        }
+
+        /** <inheritdoc /> */
+        public IDictionary<object, object> Properties
+        {
+            get { return _props ?? (_props = new Dictionary<object, object>(2)); }
+        }
+
+        /// <summary>
+        /// Clear session state.
+        /// </summary>
+        public void Clear()
+        {
+            if (_props != null)
+                _props.Clear();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStoreSessionProxy.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStoreSessionProxy.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStoreSessionProxy.cs
new file mode 100644
index 0000000..3dd7354
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStoreSessionProxy.cs
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Impl.Cache.Store
+{
+    using System.Collections.Generic;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Threading;
+    using Apache.Ignite.Core.Cache.Store;
+
+    /// <summary>
+    /// Store session proxy.
+    /// </summary>
+    [SuppressMessage("Microsoft.Design", "CA1001:TypesThatOwnDisposableFieldsShouldBeDisposable")]
+    internal class CacheStoreSessionProxy : ICacheStoreSession
+    {
+        /** Session. */
+        private readonly ThreadLocal<CacheStoreSession> _target = new ThreadLocal<CacheStoreSession>();
+
+        /** <inheritdoc /> */ 
+        public string CacheName
+        {
+            get { return _target.Value.CacheName; }
+        }
+
+        /** <inheritdoc /> */ 
+        public IDictionary<object, object> Properties
+        {
+            get { return _target.Value.Properties; }
+        }
+
+        /// <summary>
+        /// Set thread-bound session.
+        /// </summary>
+        /// <param name="ses">Session.</param>
+        internal void SetSession(CacheStoreSession ses)
+        {
+            _target.Value = ses;
+        }
+
+        /// <summary>
+        /// Clear thread-bound session.
+        /// </summary>
+        internal void ClearSession()
+        {
+            _target.Value = null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
new file mode 100644
index 0000000..d26f52e
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
@@ -0,0 +1,577 @@
+/*
+ * 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.Cluster
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Linq;
+    using System.Threading;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Events;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Compute;
+    using Apache.Ignite.Core.Impl.Events;
+    using Apache.Ignite.Core.Impl.Messaging;
+    using Apache.Ignite.Core.Impl.Portable;
+    using Apache.Ignite.Core.Impl.Portable.Metadata;
+    using Apache.Ignite.Core.Impl.Services;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+    using Apache.Ignite.Core.Messaging;
+    using Apache.Ignite.Core.Portable;
+    using Apache.Ignite.Core.Services;
+    using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils;
+
+    /// <summary>
+    /// Ignite projection implementation.
+    /// </summary>
+    internal class ClusterGroupImpl : PlatformTarget, IClusterGroupEx
+    {
+        /** Attribute: platform. */
+        private const string AttrPlatform = "org.apache.ignite.platform";
+
+        /** Platform. */
+        private const string Platform = "dotnet";
+
+        /** Initial topver; invalid from Java perspective, so update will be triggered when this value is met. */
+        private const int TopVerInit = 0;
+
+        /** */
+        private const int OpAllMetadata = 1;
+
+        /** */
+        private const int OpForAttribute = 2;
+
+        /** */
+        private const int OpForCache = 3;
+
+        /** */
+        private const int OpForClient = 4;
+
+        /** */
+        private const int OpForData = 5;
+
+        /** */
+        private const int OpForHost = 6;
+
+        /** */
+        private const int OpForNodeIds = 7;
+
+        /** */
+        private const int OpMetadata = 8;
+
+        /** */
+        private const int OpMetrics = 9;
+
+        /** */
+        private const int OpMetricsFiltered = 10;
+
+        /** */
+        private const int OpNodeMetrics = 11;
+
+        /** */
+        private const int OpNodes = 12;
+
+        /** */
+        private const int OpPingNode = 13;
+
+        /** */
+        private const int OpTopology = 14;
+
+        /** Initial Ignite instance. */
+        private readonly Ignite _ignite;
+        
+        /** Predicate. */
+        private readonly Func<IClusterNode, bool> _pred;
+
+        /** Topology version. */
+        [SuppressMessage("Microsoft.Performance", "CA1805:DoNotInitializeUnnecessarily")]
+        private long _topVer = TopVerInit;
+
+        /** Nodes for the given topology version. */
+        private volatile IList<IClusterNode> _nodes;
+
+        /** Processor. */
+        private readonly IUnmanagedTarget _proc;
+
+        /** Compute. */
+        private readonly Lazy<Compute> _comp;
+
+        /** Messaging. */
+        private readonly Lazy<Messaging> _msg;
+
+        /** Events. */
+        private readonly Lazy<Events> _events;
+
+        /** Services. */
+        private readonly Lazy<IServices> _services;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="proc">Processor.</param>
+        /// <param name="target">Target.</param>
+        /// <param name="marsh">Marshaller.</param>
+        /// <param name="ignite">Grid.</param>
+        /// <param name="pred">Predicate.</param>
+        public ClusterGroupImpl(IUnmanagedTarget proc, IUnmanagedTarget target, PortableMarshaller marsh,
+            Ignite ignite, Func<IClusterNode, bool> pred)
+            : base(target, marsh)
+        {
+            _proc = proc;
+            _ignite = ignite;
+            _pred = pred;
+
+            _comp = new Lazy<Compute>(() => 
+                new Compute(new ComputeImpl(UU.ProcessorCompute(proc, target), marsh, this, false)));
+
+            _msg = new Lazy<Messaging>(() => new Messaging(UU.ProcessorMessage(proc, target), marsh, this));
+
+            _events = new Lazy<Events>(() => new Events(UU.ProcessorEvents(proc, target), marsh, this));
+
+            _services = new Lazy<IServices>(() => 
+                new Services(UU.ProcessorServices(proc, target), marsh, this, false, false));
+        }
+
+        /** <inheritDoc /> */
+        public IIgnite Ignite
+        {
+            get { return _ignite; }
+        }
+
+        /** <inheritDoc /> */
+        public ICompute Compute()
+        {
+            return _comp.Value;
+        }
+
+        /** <inheritDoc /> */
+        public IClusterGroup ForNodes(IEnumerable<IClusterNode> nodes)
+        {
+            IgniteArgumentCheck.NotNull(nodes, "nodes");
+
+            return ForNodeIds0(nodes, node => node.Id);
+        }
+
+        /** <inheritDoc /> */
+        public IClusterGroup ForNodes(params IClusterNode[] nodes)
+        {
+            IgniteArgumentCheck.NotNull(nodes, "nodes");
+
+            return ForNodeIds0(nodes, node => node.Id);
+        }
+
+        /** <inheritDoc /> */
+        public IClusterGroup ForNodeIds(IEnumerable<Guid> ids)
+        {
+            IgniteArgumentCheck.NotNull(ids, "ids");
+
+            return ForNodeIds0(ids, null);
+        }
+
+        /** <inheritDoc /> */
+        public IClusterGroup ForNodeIds(params Guid[] ids)
+        {
+            IgniteArgumentCheck.NotNull(ids, "ids");
+
+            return ForNodeIds0(ids, null);
+        }
+
+        /// <summary>
+        /// Internal routine to get projection for specific node IDs.
+        /// </summary>
+        /// <param name="items">Items.</param>
+        /// <param name="func">Function to transform item to Guid (optional).</param>
+        /// <returns></returns>
+        private IClusterGroup ForNodeIds0<T>(IEnumerable<T> items, Func<T, Guid> func)
+        {
+            Debug.Assert(items != null);
+
+            IUnmanagedTarget prj = DoProjetionOutOp(OpForNodeIds, writer =>
+            {
+                WriteEnumerable(writer, items, func);
+            });
+            
+            return GetClusterGroup(prj);
+        }
+
+        /** <inheritDoc /> */
+        public IClusterGroup ForPredicate(Func<IClusterNode, bool> p)
+        {
+            var newPred = _pred == null ? p : node => _pred(node) && p(node);
+
+            return new ClusterGroupImpl(_proc, Target, Marshaller, _ignite, newPred);
+        }
+
+        /** <inheritDoc /> */
+        public IClusterGroup ForAttribute(string name, string val)
+        {
+            IgniteArgumentCheck.NotNull(name, "name");
+
+            IUnmanagedTarget prj = DoProjetionOutOp(OpForAttribute, writer =>
+            {
+                writer.WriteString(name);
+                writer.WriteString(val);
+            });
+
+            return GetClusterGroup(prj);
+        }
+
+        /// <summary>
+        /// Creates projection with a specified op.
+        /// </summary>
+        /// <param name="name">Cache name to include into projection.</param>
+        /// <param name="op">Operation id.</param>
+        /// <returns>
+        /// Projection over nodes that have specified cache running.
+        /// </returns>
+        private IClusterGroup ForCacheNodes(string name, int op)
+        {
+            IUnmanagedTarget prj = DoProjetionOutOp(op, writer =>
+            {
+                writer.WriteString(name);
+            });
+
+            return GetClusterGroup(prj);
+        }
+
+        /** <inheritDoc /> */
+        public IClusterGroup ForCacheNodes(string name)
+        {
+            return ForCacheNodes(name, OpForCache);
+        }
+
+        /** <inheritDoc /> */
+        public IClusterGroup ForDataNodes(string name)
+        {
+            return ForCacheNodes(name, OpForData);
+        }
+
+        /** <inheritDoc /> */
+        public IClusterGroup ForClientNodes(string name)
+        {
+            return ForCacheNodes(name, OpForClient);
+        }
+
+        /** <inheritDoc /> */
+        public IClusterGroup ForRemotes()
+        {
+            return GetClusterGroup(UU.ProjectionForRemotes(Target));
+        }
+
+        /** <inheritDoc /> */
+        public IClusterGroup ForHost(IClusterNode node)
+        {
+            IgniteArgumentCheck.NotNull(node, "node");
+
+            IUnmanagedTarget prj = DoProjetionOutOp(OpForHost, writer =>
+            {
+                writer.WriteGuid(node.Id);
+            });    
+                    
+            return GetClusterGroup(prj);
+        }
+
+        /** <inheritDoc /> */
+        public IClusterGroup ForRandom()
+        {
+            return GetClusterGroup(UU.ProjectionForRandom(Target));
+        }
+
+        /** <inheritDoc /> */
+        public IClusterGroup ForOldest()
+        {
+            return GetClusterGroup(UU.ProjectionForOldest(Target));
+        }
+
+        /** <inheritDoc /> */
+        public IClusterGroup ForYoungest()
+        {
+            return GetClusterGroup(UU.ProjectionForYoungest(Target));
+        }
+
+        /** <inheritDoc /> */
+        public IClusterGroup ForDotNet()
+        {
+            return ForAttribute(AttrPlatform, Platform);
+        }
+
+        /** <inheritDoc /> */
+        public ICollection<IClusterNode> Nodes()
+        {
+            return RefreshNodes();
+        }
+
+        /** <inheritDoc /> */
+        public IClusterNode Node(Guid id)
+        {
+            return Nodes().FirstOrDefault(node => node.Id == id);
+        }
+
+        /** <inheritDoc /> */
+        public IClusterNode Node()
+        {
+            return Nodes().FirstOrDefault();
+        }
+
+        /** <inheritDoc /> */
+        public IClusterMetrics Metrics()
+        {
+            if (_pred == null)
+            {
+                return DoInOp(OpMetrics, stream =>
+                {
+                    IPortableRawReader reader = Marshaller.StartUnmarshal(stream, false);
+
+                    return reader.ReadBoolean() ? new ClusterMetricsImpl(reader) : null;
+                });
+            }
+            return DoOutInOp(OpMetricsFiltered, writer =>
+            {
+                WriteEnumerable(writer, Nodes().Select(node => node.Id));
+            }, stream =>
+            {
+                IPortableRawReader reader = Marshaller.StartUnmarshal(stream, false);
+
+                return reader.ReadBoolean() ? new ClusterMetricsImpl(reader) : null;
+            });
+        }
+
+        /** <inheritDoc /> */
+        public IMessaging Message()
+        {
+            return _msg.Value;
+        }
+
+        /** <inheritDoc /> */
+        public IEvents Events()
+        {
+            return _events.Value;
+        }
+
+        /** <inheritDoc /> */
+        public IServices Services()
+        {
+            return _services.Value;
+        }
+
+        /// <summary>
+        /// Pings a remote node.
+        /// </summary>
+        /// <param name="nodeId">ID of a node to ping.</param>
+        /// <returns>True if node for a given ID is alive, false otherwise.</returns>
+        internal bool PingNode(Guid nodeId)
+        {
+            return DoOutOp(OpPingNode, nodeId) == True;
+        }
+
+        /// <summary>
+        /// Predicate (if any).
+        /// </summary>
+        public Func<IClusterNode, bool> Predicate
+        {
+            get { return _pred; }
+        }
+
+        /// <summary>
+        /// Refresh cluster node metrics.
+        /// </summary>
+        /// <param name="nodeId">Node</param>
+        /// <param name="lastUpdateTime"></param>
+        /// <returns></returns>
+        internal ClusterMetricsImpl RefreshClusterNodeMetrics(Guid nodeId, long lastUpdateTime)
+        {
+            return DoOutInOp(OpNodeMetrics, writer =>
+                {
+                    writer.WriteGuid(nodeId);
+                    writer.WriteLong(lastUpdateTime);
+                }, stream =>
+                {
+                    IPortableRawReader reader = Marshaller.StartUnmarshal(stream, false);
+
+                    return reader.ReadBoolean() ? new ClusterMetricsImpl(reader) : null;
+                }
+            );
+        }
+
+        /// <summary>
+        /// Gets a topology by version. Returns null if topology history storage doesn't contain 
+        /// specified topology version (history currently keeps the last 1000 snapshots).
+        /// </summary>
+        /// <param name="version">Topology version.</param>
+        /// <returns>Collection of Ignite nodes which represented by specified topology version, 
+        /// if it is present in history storage, {@code null} otherwise.</returns>
+        /// <exception cref="IgniteException">If underlying SPI implementation does not support 
+        /// topology history. Currently only {@link org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi}
+        /// supports topology history.</exception>
+        internal ICollection<IClusterNode> Topology(long version)
+        {
+            return DoOutInOp(OpTopology, writer => writer.WriteLong(version), 
+                input => IgniteUtils.ReadNodes(Marshaller.StartUnmarshal(input)));
+        }
+
+        /// <summary>
+        /// Topology version.
+        /// </summary>
+        internal long TopologyVersion
+        {
+            get
+            {
+                RefreshNodes();
+
+                return Interlocked.Read(ref _topVer);
+            }
+        }
+
+        /// <summary>
+        /// Update topology.
+        /// </summary>
+        /// <param name="newTopVer">New topology version.</param>
+        /// <param name="newNodes">New nodes.</param>
+        internal void UpdateTopology(long newTopVer, List<IClusterNode> newNodes)
+        {
+            lock (this)
+            {
+                // If another thread already advanced topology version further, we still
+                // can safely return currently received nodes, but we will not assign them.
+                if (_topVer < newTopVer)
+                {
+                    Interlocked.Exchange(ref _topVer, newTopVer);
+
+                    _nodes = newNodes.AsReadOnly();
+                }
+            }
+        }
+
+        /// <summary>
+        /// Get current nodes without refreshing the topology.
+        /// </summary>
+        /// <returns>Current nodes.</returns>
+        internal IList<IClusterNode> NodesNoRefresh()
+        {
+            return _nodes;
+        }
+
+        /// <summary>
+        /// Creates new Cluster Group from given native projection.
+        /// </summary>
+        /// <param name="prj">Native projection.</param>
+        /// <returns>New cluster group.</returns>
+        private IClusterGroup GetClusterGroup(IUnmanagedTarget prj)
+        {
+            return new ClusterGroupImpl(_proc, prj, Marshaller, _ignite, _pred);
+        }
+
+        /// <summary>
+        /// Refresh projection nodes.
+        /// </summary>
+        /// <returns>Nodes.</returns>
+        private IList<IClusterNode> RefreshNodes()
+        {
+            long oldTopVer = Interlocked.Read(ref _topVer);
+
+            List<IClusterNode> newNodes = null;
+
+            DoOutInOp(OpNodes, writer =>
+            {
+                writer.WriteLong(oldTopVer);
+            }, input =>
+            {
+                PortableReaderImpl reader = Marshaller.StartUnmarshal(input);
+
+                if (reader.ReadBoolean())
+                {
+                    // Topology has been updated.
+                    long newTopVer = reader.ReadLong();
+
+                    newNodes = IgniteUtils.ReadNodes(reader, _pred);
+
+                    UpdateTopology(newTopVer, newNodes);
+                }
+            });
+
+            if (newNodes != null)
+                return newNodes;
+            
+            // No topology changes.
+            Debug.Assert(_nodes != null, "At least one topology update should have occurred.");
+
+            return _nodes;
+        }
+        
+        /// <summary>
+        /// Perform synchronous out operation returning value.
+        /// </summary>
+        /// <param name="type">Operation type.</param>
+        /// <param name="action">Action.</param>
+        /// <returns>Native projection.</returns>
+        private IUnmanagedTarget DoProjetionOutOp(int type, Action<PortableWriterImpl> action)
+        {
+            using (var stream = IgniteManager.Memory.Allocate().Stream())
+            {
+                var writer = Marshaller.StartMarshal(stream);
+
+                action(writer);
+
+                FinishMarshal(writer);
+
+                return UU.ProjectionOutOpRet(Target, type, stream.SynchronizeOutput());
+            }
+        }
+        
+        /** <inheritDoc /> */
+        public IPortableMetadata Metadata(int typeId)
+        {
+            return DoOutInOp<IPortableMetadata>(OpMetadata, 
+                writer =>
+                {
+                    writer.WriteInt(typeId);
+                },
+                stream =>
+                {
+                    PortableReaderImpl reader = Marshaller.StartUnmarshal(stream, false);
+
+                    return reader.ReadBoolean() ? new PortableMetadataImpl(reader) : null;
+                }
+            );
+        }
+
+        /// <summary>
+        /// Gets metadata for all known types.
+        /// </summary>
+        public List<IPortableMetadata> Metadata()
+        {
+            return DoInOp(OpAllMetadata, s =>
+            {
+                var reader = Marshaller.StartUnmarshal(s);
+
+                var size = reader.ReadInt();
+
+                var res = new List<IPortableMetadata>(size);
+
+                for (var i = 0; i < size; i++)
+                    res.Add(reader.ReadBoolean() ? new PortableMetadataImpl(reader) : null);
+
+                return res;
+            });
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterMetricsImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterMetricsImpl.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterMetricsImpl.cs
new file mode 100644
index 0000000..664a1f1
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterMetricsImpl.cs
@@ -0,0 +1,292 @@
+/*
+ * 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.Cluster
+{
+    using System;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Cluster metrics implementation.
+    /// </summary>
+    internal class ClusterMetricsImpl : IClusterMetrics
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ClusterMetricsImpl"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        public ClusterMetricsImpl(IPortableRawReader reader)
+        {
+            LastUpdateTimeRaw = reader.ReadLong();
+
+            DateTime? lastUpdateTime0 = reader.ReadDate();
+
+            LastUpdateTime = lastUpdateTime0 ?? default(DateTime);
+            MaximumActiveJobs = reader.ReadInt();
+            CurrentActiveJobs = reader.ReadInt();
+            AverageActiveJobs = reader.ReadFloat();
+            MaximumWaitingJobs = reader.ReadInt();
+
+            CurrentWaitingJobs = reader.ReadInt();
+            AverageWaitingJobs = reader.ReadFloat();
+            MaximumRejectedJobs = reader.ReadInt();
+            CurrentRejectedJobs = reader.ReadInt();
+            AverageRejectedJobs = reader.ReadFloat();
+
+            TotalRejectedJobs = reader.ReadInt();
+            MaximumCancelledJobs = reader.ReadInt();
+            CurrentCancelledJobs = reader.ReadInt();
+            AverageCancelledJobs = reader.ReadFloat();
+            TotalCancelledJobs = reader.ReadInt();
+
+            TotalExecutedJobs = reader.ReadInt();
+            MaximumJobWaitTime = reader.ReadLong();
+            CurrentJobWaitTime = reader.ReadLong();
+            AverageJobWaitTime = reader.ReadDouble();
+            MaximumJobExecuteTime = reader.ReadLong();
+
+            CurrentJobExecuteTime = reader.ReadLong();
+            AverageJobExecuteTime = reader.ReadDouble();
+            TotalExecutedTasks = reader.ReadInt();
+            TotalIdleTime = reader.ReadLong();
+            CurrentIdleTime = reader.ReadLong();
+
+            TotalCpus = reader.ReadInt();
+            CurrentCpuLoad = reader.ReadDouble();
+            AverageCpuLoad = reader.ReadDouble();
+            CurrentGcCpuLoad = reader.ReadDouble();
+            HeapMemoryInitialized = reader.ReadLong();
+
+            HeapMemoryUsed = reader.ReadLong();
+            HeapMemoryCommitted = reader.ReadLong();
+            HeapMemoryMaximum = reader.ReadLong();
+            HeapMemoryTotal = reader.ReadLong();
+            NonHeapMemoryInitialized = reader.ReadLong();
+
+            NonHeapMemoryUsed = reader.ReadLong();
+            NonHeapMemoryCommitted = reader.ReadLong();
+            NonHeapMemoryMaximum = reader.ReadLong();
+            NonHeapMemoryTotal = reader.ReadLong();
+            UpTime = reader.ReadLong();
+
+            DateTime? startTime0 = reader.ReadDate();
+
+            StartTime = startTime0 ?? default(DateTime);
+
+            DateTime? nodeStartTime0 = reader.ReadDate();
+
+            NodeStartTime = nodeStartTime0 ?? default(DateTime);
+
+            CurrentThreadCount = reader.ReadInt();
+            MaximumThreadCount = reader.ReadInt();
+            TotalStartedThreadCount = reader.ReadLong();
+            CurrentDaemonThreadCount = reader.ReadInt();
+            LastDataVersion = reader.ReadLong();
+
+            SentMessagesCount = reader.ReadInt();
+            SentBytesCount = reader.ReadLong();
+            ReceivedMessagesCount = reader.ReadInt();
+            ReceivedBytesCount = reader.ReadLong();
+            OutboundMessagesQueueSize = reader.ReadInt();
+
+            TotalNodes = reader.ReadInt();
+        }
+
+        /// <summary>
+        /// Last update time in raw format.
+        /// </summary>
+        internal long LastUpdateTimeRaw { get; set; }
+
+        /** <inheritDoc /> */
+        public DateTime LastUpdateTime { get; private set; }
+
+        /** <inheritDoc /> */
+        public int MaximumActiveJobs { get; private set; }
+
+        /** <inheritDoc /> */
+        public int CurrentActiveJobs { get; private set; }
+
+        /** <inheritDoc /> */
+        public float AverageActiveJobs { get; private set; }
+
+        /** <inheritDoc /> */
+        public int MaximumWaitingJobs { get; private set; }
+
+        /** <inheritDoc /> */
+        public int CurrentWaitingJobs { get; private set; }
+
+        /** <inheritDoc /> */
+        public float AverageWaitingJobs { get; private set; }
+
+        /** <inheritDoc /> */
+        public int MaximumRejectedJobs { get; private set; }
+
+        /** <inheritDoc /> */
+        public int CurrentRejectedJobs { get; private set; }
+
+        /** <inheritDoc /> */
+        public float AverageRejectedJobs { get; private set; }
+
+        /** <inheritDoc /> */
+        public int TotalRejectedJobs { get; private set; }
+
+        /** <inheritDoc /> */
+        public int MaximumCancelledJobs { get; private set; }
+
+        /** <inheritDoc /> */
+        public int CurrentCancelledJobs { get; private set; }
+
+        /** <inheritDoc /> */
+        public float AverageCancelledJobs { get; private set; }
+
+        /** <inheritDoc /> */
+        public int TotalCancelledJobs { get; private set; }
+
+        /** <inheritDoc /> */
+        public int TotalExecutedJobs { get; private set; }
+
+        /** <inheritDoc /> */
+        public long MaximumJobWaitTime { get; private set; }
+
+        /** <inheritDoc /> */
+        public long CurrentJobWaitTime { get; private set; }
+
+        /** <inheritDoc /> */
+        public double AverageJobWaitTime { get; private set; }
+
+        /** <inheritDoc /> */
+        public long MaximumJobExecuteTime { get; private set; }
+
+        /** <inheritDoc /> */
+        public long CurrentJobExecuteTime { get; private set; }
+
+        /** <inheritDoc /> */
+        public double AverageJobExecuteTime { get; private set; }
+
+        /** <inheritDoc /> */
+        public int TotalExecutedTasks { get; private set; }
+
+        /** <inheritDoc /> */
+        public long TotalBusyTime
+        {
+            get { return UpTime - TotalIdleTime; }
+        }
+
+        /** <inheritDoc /> */
+        public long TotalIdleTime { get; private set; }
+
+        /** <inheritDoc /> */
+        public long CurrentIdleTime { get; private set; }
+
+        /** <inheritDoc /> */
+        public float BusyTimePercentage
+        {
+            get { return 1 - IdleTimePercentage; }
+        }
+
+        /** <inheritDoc /> */
+        public float IdleTimePercentage
+        {
+            get { return TotalIdleTime / (float) UpTime; }
+        }
+
+        /** <inheritDoc /> */
+        public int TotalCpus { get; private set; }
+
+        /** <inheritDoc /> */
+        public double CurrentCpuLoad { get; private set; }
+
+        /** <inheritDoc /> */
+        public double AverageCpuLoad { get; private set; }
+
+        /** <inheritDoc /> */
+        public double CurrentGcCpuLoad { get; private set; }
+
+        /** <inheritDoc /> */
+        public long HeapMemoryInitialized { get; private set; }
+
+        /** <inheritDoc /> */
+        public long HeapMemoryUsed { get; private set; }
+
+        /** <inheritDoc /> */
+        public long HeapMemoryCommitted { get; private set; }
+
+        /** <inheritDoc /> */
+        public long HeapMemoryMaximum { get; private set; }
+
+        /** <inheritDoc /> */
+        public long HeapMemoryTotal { get; private set; }
+
+        /** <inheritDoc /> */
+        public long NonHeapMemoryInitialized { get; private set; }
+
+        /** <inheritDoc /> */
+        public long NonHeapMemoryUsed { get; private set; }
+
+        /** <inheritDoc /> */
+        public long NonHeapMemoryCommitted { get; private set; }
+
+        /** <inheritDoc /> */
+        public long NonHeapMemoryMaximum { get; private set; }
+
+        /** <inheritDoc /> */
+        public long NonHeapMemoryTotal { get; private set; }
+
+        /** <inheritDoc /> */
+        public long UpTime { get; private set; }
+
+        /** <inheritDoc /> */
+        public DateTime StartTime { get; private set; }
+
+        /** <inheritDoc /> */
+        public DateTime NodeStartTime { get; private set; }
+
+        /** <inheritDoc /> */
+        public int CurrentThreadCount { get; private set; }
+
+        /** <inheritDoc /> */
+        public int MaximumThreadCount { get; private set; }
+
+        /** <inheritDoc /> */
+        public long TotalStartedThreadCount { get; private set; }
+
+        /** <inheritDoc /> */
+        public int CurrentDaemonThreadCount { get; private set; }
+
+        /** <inheritDoc /> */
+        public long LastDataVersion { get; private set; }
+
+        /** <inheritDoc /> */
+        public int SentMessagesCount { get; private set; }
+
+        /** <inheritDoc /> */
+        public long SentBytesCount { get; private set; }
+
+        /** <inheritDoc /> */
+        public int ReceivedMessagesCount { get; private set; }
+
+        /** <inheritDoc /> */
+        public long ReceivedBytesCount { get; private set; }
+
+        /** <inheritDoc /> */
+        public int OutboundMessagesQueueSize { get; private set; }
+
+        /** <inheritDoc /> */
+        public int TotalNodes { get; private set; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterNodeImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterNodeImpl.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterNodeImpl.cs
new file mode 100644
index 0000000..59373a2
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterNodeImpl.cs
@@ -0,0 +1,221 @@
+/*
+ * 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.Cluster
+{
+    using System;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Impl.Collections;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Cluster node implementation.
+    /// </summary>
+    internal class ClusterNodeImpl : IClusterNode
+    {
+        /** Node ID. */
+        private readonly Guid _id;
+
+        /** Attributes. */
+        private readonly IDictionary<string, object> _attrs;
+
+        /** Addresses. */
+        private readonly ICollection<string> _addrs;
+
+        /** Hosts. */
+        private readonly ICollection<string> _hosts;
+
+        /** Order. */
+        private readonly long _order;
+
+        /** Local flag. */
+        private readonly bool _local;
+
+        /** Daemon flag. */
+        private readonly bool _daemon;
+
+        /** Metrics. */
+        private volatile ClusterMetricsImpl _metrics;
+        
+        /** Ignite reference. */
+        private WeakReference _igniteRef;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ClusterNodeImpl"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        public ClusterNodeImpl(IPortableRawReader reader)
+        {
+            _id = reader.ReadGuid() ?? default(Guid);
+
+            _attrs = reader.ReadGenericDictionary<string, object>().AsReadOnly();
+            _addrs = reader.ReadGenericCollection<string>().AsReadOnly();
+            _hosts = reader.ReadGenericCollection<string>().AsReadOnly();
+            _order = reader.ReadLong();
+            _local = reader.ReadBoolean();
+            _daemon = reader.ReadBoolean();
+
+            _metrics = reader.ReadBoolean() ? new ClusterMetricsImpl(reader) : null;
+        }
+
+        /** <inheritDoc /> */
+        public Guid Id
+        {
+            get { return _id; }
+        }
+
+        /** <inheritDoc /> */
+        public T Attribute<T>(string name)
+        {
+            IgniteArgumentCheck.NotNull(name, "name");
+
+            return (T)_attrs[name];
+        }
+
+        /** <inheritDoc /> */
+        public bool TryGetAttribute<T>(string name, out T attr)
+        {
+            IgniteArgumentCheck.NotNull(name, "name");
+
+            object val;
+
+            if (_attrs.TryGetValue(name, out val))
+            {
+                attr = (T)val;
+
+                return true;
+            }
+            attr = default(T);
+
+            return false;
+        }
+
+        /** <inheritDoc /> */
+        public IDictionary<string, object> Attributes()
+        {
+            return _attrs;
+        }
+
+        /** <inheritDoc /> */
+        public ICollection<string> Addresses
+        {
+            get
+            {
+                return _addrs;
+            }
+        }
+
+        /** <inheritDoc /> */
+        public ICollection<string> HostNames
+        {
+            get
+            {
+                return _hosts;
+            }
+        }
+
+        /** <inheritDoc /> */
+        public long Order
+        {
+            get
+            {
+                return _order;
+            }
+        }
+
+        /** <inheritDoc /> */
+        public bool IsLocal
+        {
+            get
+            {
+                return _local;
+            }
+        }
+
+        /** <inheritDoc /> */
+        public bool IsDaemon
+        {
+            get
+            {
+                return _daemon;
+            }
+        }
+
+        /** <inheritDoc /> */
+        public IClusterMetrics Metrics()
+        {
+            var ignite = (Ignite)_igniteRef.Target;
+
+            if (ignite == null)
+                return _metrics;
+
+            ClusterMetricsImpl oldMetrics = _metrics;
+
+            long lastUpdateTime = oldMetrics.LastUpdateTimeRaw;
+
+            ClusterMetricsImpl newMetrics = ignite.ClusterGroup.RefreshClusterNodeMetrics(_id, lastUpdateTime);
+
+            if (newMetrics != null)
+            {
+                lock (this)
+                {
+                    if (_metrics.LastUpdateTime < newMetrics.LastUpdateTime)
+                        _metrics = newMetrics;
+                }
+
+                return newMetrics;
+            }
+
+            return oldMetrics;
+        }
+        
+        /** <inheritDoc /> */
+        public override string ToString()
+        {
+            return "GridNode [id=" + Id + ']';
+        }
+
+        /** <inheritDoc /> */
+        public override bool Equals(object obj)
+        {
+            ClusterNodeImpl node = obj as ClusterNodeImpl;
+
+            if (node != null)
+                return _id.Equals(node._id);
+
+            return false;
+        }
+
+        /** <inheritDoc /> */
+        public override int GetHashCode()
+        {
+            // ReSharper disable once NonReadonlyMemberInGetHashCode
+            return _id.GetHashCode();
+        }
+
+        /// <summary>
+        /// Initializes this instance with a grid.
+        /// </summary>
+        /// <param name="grid">The grid.</param>
+        internal void Init(Ignite grid)
+        {
+            _igniteRef = new WeakReference(grid);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cluster/IClusterGroupEx.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cluster/IClusterGroupEx.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cluster/IClusterGroupEx.cs
new file mode 100644
index 0000000..554eb0a
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Cluster/IClusterGroupEx.cs
@@ -0,0 +1,35 @@
+/*
+ * 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.Cluster
+{
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// 
+    /// </summary>
+    internal interface IClusterGroupEx : IClusterGroup
+    {
+        /// <summary>
+        /// Gets protable metadata for type.
+        /// </summary>
+        /// <param name="typeId">Type ID.</param>
+        /// <returns>Metadata.</returns>
+        IPortableMetadata Metadata(int typeId);
+    }
+}


[07/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
new file mode 100644
index 0000000..39b6b24
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
@@ -0,0 +1,961 @@
+/*
+ * 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.Collections.Concurrent;
+    using System.Collections.Generic;
+    using System.Linq;
+    using System.Threading;
+    using System.Threading.Tasks;
+    using Apache.Ignite.Core.Cache.Query;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Events;
+    using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.Impl.Events;
+    using Apache.Ignite.Core.Portable;
+    using Apache.Ignite.Core.Tests.Compute;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// <see cref="IEvents"/> tests.
+    /// </summary>
+    public class EventsTest
+    {
+        /** */
+        private IIgnite _grid1;
+
+        /** */
+        private IIgnite _grid2;
+
+        /** */
+        private IIgnite _grid3;
+
+        /** */
+        private IIgnite[] _grids;
+        
+        /** */
+        public static int IdGen;
+
+        [TestFixtureTearDown]
+        public void FixtureTearDown()
+        {
+            StopGrids();
+        }
+
+        /// <summary>
+        /// Executes before each test.
+        /// </summary>
+        [SetUp]
+        public void SetUp()
+        {
+            StartGrids();
+            EventsTestHelper.ListenResult = true;
+        }
+
+        /// <summary>
+        /// Executes after each test.
+        /// </summary>
+        [TearDown]
+        public virtual void TearDown()
+        {
+            try
+            {
+                TestUtils.AssertHandleRegistryIsEmpty(1000, _grid1, _grid2, _grid3);
+            }
+            catch (Exception)
+            {
+                // Restart grids to cleanup
+                StopGrids();
+
+                throw;
+            }
+            finally
+            {
+                EventsTestHelper.AssertFailures();
+
+                if (TestContext.CurrentContext.Test.Name.StartsWith("TestEventTypes"))
+                    StopGrids(); // clean events for other tests
+            }
+        }
+
+        /// <summary>
+        /// Tests enable/disable of event types.
+        /// </summary>
+        [Test]
+        public void TestEnableDisable()
+        {
+            var events = _grid1.Events();
+
+            Assert.AreEqual(0, events.GetEnabledEvents().Length);
+
+            Assert.IsFalse(EventType.EvtsCache.Any(events.IsEnabled));
+
+            events.EnableLocal(EventType.EvtsCache);
+
+            Assert.AreEqual(EventType.EvtsCache, events.GetEnabledEvents());
+
+            Assert.IsTrue(EventType.EvtsCache.All(events.IsEnabled));
+
+            events.EnableLocal(EventType.EvtsTaskExecution);
+
+            events.DisableLocal(EventType.EvtsCache);
+
+            Assert.AreEqual(EventType.EvtsTaskExecution, events.GetEnabledEvents());
+        }
+
+        /// <summary>
+        /// Tests LocalListen.
+        /// </summary>
+        [Test]
+        public void TestLocalListen()
+        {
+            var events = _grid1.Events();
+            var listener = EventsTestHelper.GetListener();
+            var eventType = EventType.EvtsTaskExecution;
+
+            events.EnableLocal(eventType);
+
+            events.LocalListen(listener, eventType);
+
+            CheckSend(3);  // 3 events per task * 3 grids
+
+            // Check unsubscription for specific event
+            events.StopLocalListen(listener, EventType.EvtTaskReduced);
+
+            CheckSend(2);
+
+            // Unsubscribe from all events
+            events.StopLocalListen(listener);
+
+            CheckNoEvent();
+
+            // Check unsubscription by filter
+            events.LocalListen(listener, EventType.EvtTaskReduced);
+
+            CheckSend();
+
+            EventsTestHelper.ListenResult = false;
+
+            CheckSend();  // one last event will be received for each listener
+
+            CheckNoEvent();
+        }
+
+        /// <summary>
+        /// Tests LocalListen.
+        /// </summary>
+        [Test]
+        [Ignore("IGNITE-879")]
+        public void TestLocalListenRepeatedSubscription()
+        {
+            var events = _grid1.Events();
+            var listener = EventsTestHelper.GetListener();
+            var eventType = EventType.EvtsTaskExecution;
+
+            events.EnableLocal(eventType);
+
+            events.LocalListen(listener, eventType);
+
+            CheckSend(3);  // 3 events per task * 3 grids
+
+            events.LocalListen(listener, eventType);
+            events.LocalListen(listener, eventType);
+
+            CheckSend(9);
+
+            events.StopLocalListen(listener, eventType);
+
+            CheckSend(6);
+
+            events.StopLocalListen(listener, eventType);
+
+            CheckSend(3);
+            
+            events.StopLocalListen(listener, eventType);
+
+            CheckNoEvent();
+        }
+
+        /// <summary>
+        /// Tests all available event types/classes.
+        /// </summary>
+        [Test, TestCaseSource("TestCases")]
+        public void TestEventTypes(EventTestCase testCase)
+        {
+            var events = _grid1.Events();
+
+            events.EnableLocal(testCase.EventType);
+
+            var listener = EventsTestHelper.GetListener();
+
+            events.LocalListen(listener, testCase.EventType);
+
+            EventsTestHelper.ClearReceived(testCase.EventCount);
+
+            testCase.GenerateEvent(_grid1);
+
+            EventsTestHelper.VerifyReceive(testCase.EventCount, testCase.EventObjectType, testCase.EventType);
+
+            if (testCase.VerifyEvents != null)
+                testCase.VerifyEvents(EventsTestHelper.ReceivedEvents.Reverse(), _grid1);
+
+            // Check stop
+            events.StopLocalListen(listener);
+
+            EventsTestHelper.ClearReceived(0);
+
+            testCase.GenerateEvent(_grid1);
+
+            Thread.Sleep(EventsTestHelper.Timeout);
+        }
+
+        /// <summary>
+        /// Test cases for TestEventTypes: type id + type + event generator.
+        /// </summary>
+        public IEnumerable<EventTestCase> TestCases
+        {
+            get
+            {
+                yield return new EventTestCase
+                {
+                    EventType = EventType.EvtsCache,
+                    EventObjectType = typeof (CacheEvent),
+                    GenerateEvent = g => g.Cache<int, int>(null).Put(1, 1),
+                    VerifyEvents = (e, g) => VerifyCacheEvents(e, g),
+                    EventCount = 1
+                };
+
+                yield return new EventTestCase
+                {
+                    EventType = EventType.EvtsTaskExecution,
+                    EventObjectType = typeof (TaskEvent),
+                    GenerateEvent = g => GenerateTaskEvent(g),
+                    VerifyEvents = (e, g) => VerifyTaskEvents(e),
+                    EventCount = 3
+                };
+
+                yield return new EventTestCase
+                {
+                    EventType = EventType.EvtsJobExecution,
+                    EventObjectType = typeof (JobEvent),
+                    GenerateEvent = g => GenerateTaskEvent(g),
+                    EventCount = 9
+                };
+
+                yield return new EventTestCase
+                {
+                    EventType = new[] {EventType.EvtCacheQueryExecuted},
+                    EventObjectType = typeof (CacheQueryExecutedEvent),
+                    GenerateEvent = g => GenerateCacheQueryEvent(g),
+                    EventCount = 1
+                };
+
+                yield return new EventTestCase
+                {
+                    EventType = new[] { EventType.EvtCacheQueryObjectRead },
+                    EventObjectType = typeof (CacheQueryReadEvent),
+                    GenerateEvent = g => GenerateCacheQueryEvent(g),
+                    EventCount = 1
+                };
+            }
+        }
+
+        /// <summary>
+        /// Tests the LocalQuery.
+        /// </summary>
+        [Test]
+        public void TestLocalQuery()
+        {
+            var events = _grid1.Events();
+
+            var eventType = EventType.EvtsTaskExecution;
+
+            events.EnableLocal(eventType);
+
+            var oldEvents = events.LocalQuery();
+
+            GenerateTaskEvent();
+
+            // "Except" works because of overridden equality
+            var qryResult = events.LocalQuery(eventType).Except(oldEvents).ToList();
+
+            Assert.AreEqual(3, qryResult.Count);
+        }
+
+        /// <summary>
+        /// Tests the WaitForLocal.
+        /// </summary>
+        [Test]
+        public void TestWaitForLocal([Values(true, false)] bool async)
+        {
+            var events = _grid1.Events();
+
+            var timeout = TimeSpan.FromSeconds(3);
+
+            if (async)
+                events = events.WithAsync();
+
+            var eventType = EventType.EvtsTaskExecution;
+
+            events.EnableLocal(eventType);
+
+            Func<Func<IEvent>, Task<IEvent>> getWaitTask;
+
+            if (async)
+                getWaitTask = func =>
+                {
+                    Assert.IsNull(func());
+                    var task = events.GetFuture<IEvent>().ToTask();
+                    GenerateTaskEvent();
+                    return task;
+                };
+            else
+                getWaitTask = func =>
+                {
+                    var task = Task.Factory.StartNew(func);
+                    Thread.Sleep(500); // allow task to start and begin waiting for events
+                    GenerateTaskEvent();
+                    return task;
+                };
+
+            // No params
+            var waitTask = getWaitTask(() => events.WaitForLocal());
+
+            waitTask.Wait(timeout);
+
+            // Event types
+            waitTask = getWaitTask(() => events.WaitForLocal(EventType.EvtTaskReduced));
+
+            Assert.IsTrue(waitTask.Wait(timeout));
+            Assert.IsInstanceOf(typeof(TaskEvent), waitTask.Result);
+            Assert.AreEqual(EventType.EvtTaskReduced, waitTask.Result.Type);
+
+            // Filter
+            waitTask = getWaitTask(() => events.WaitForLocal(
+                new EventFilter<IEvent>((g, e) => e.Type == EventType.EvtTaskReduced)));
+
+            Assert.IsTrue(waitTask.Wait(timeout));
+            Assert.IsInstanceOf(typeof(TaskEvent), waitTask.Result);
+            Assert.AreEqual(EventType.EvtTaskReduced, waitTask.Result.Type);
+
+            // Filter & types
+            waitTask = getWaitTask(() => events.WaitForLocal(
+                new EventFilter<IEvent>((g, e) => e.Type == EventType.EvtTaskReduced), EventType.EvtTaskReduced));
+
+            Assert.IsTrue(waitTask.Wait(timeout));
+            Assert.IsInstanceOf(typeof(TaskEvent), waitTask.Result);
+            Assert.AreEqual(EventType.EvtTaskReduced, waitTask.Result.Type);
+        }
+
+        /// <summary>
+        /// Tests RemoteListen.
+        /// </summary>
+        [Test]
+        public void TestRemoteListen(
+            [Values(true, false)] bool async, 
+            [Values(true, false)] bool portable,
+            [Values(true, false)] bool autoUnsubscribe)
+        {
+            foreach (var g in _grids)
+            {
+                g.Events().EnableLocal(EventType.EvtsJobExecution);
+                g.Events().EnableLocal(EventType.EvtsTaskExecution);
+            }
+
+            var events = _grid1.Events();
+
+            var expectedType = EventType.EvtJobStarted;
+
+            var remoteFilter = portable 
+                ?  (IEventFilter<IEvent>) new RemoteEventPortableFilter(expectedType) 
+                :  new RemoteEventFilter(expectedType);
+
+            var localListener = EventsTestHelper.GetListener();
+
+            if (async)
+                events = events.WithAsync();
+
+            var listenId = events.RemoteListen(localListener: localListener, remoteFilter: remoteFilter,
+                autoUnsubscribe: autoUnsubscribe);
+
+            if (async)
+                listenId = events.GetFuture<Guid>().Get();
+
+            CheckSend(3, typeof(JobEvent), expectedType);
+
+            _grid3.Events().DisableLocal(EventType.EvtsJobExecution);
+
+            CheckSend(2, typeof(JobEvent), expectedType);
+
+            events.StopRemoteListen(listenId);
+
+            if (async)
+                events.GetFuture().Get();
+
+            CheckNoEvent();
+
+            // Check unsubscription with listener
+            events.RemoteListen(localListener: localListener, remoteFilter: remoteFilter,
+                autoUnsubscribe: autoUnsubscribe);
+
+            if (async)
+                events.GetFuture<Guid>().Get();
+
+            CheckSend(2, typeof(JobEvent), expectedType);
+
+            EventsTestHelper.ListenResult = false;
+
+            CheckSend(1, typeof(JobEvent), expectedType);  // one last event
+
+            CheckNoEvent();
+        }
+
+        /// <summary>
+        /// Tests RemoteQuery.
+        /// </summary>
+        [Test]
+        public void TestRemoteQuery([Values(true, false)] bool async)
+        {
+            foreach (var g in _grids)
+                g.Events().EnableLocal(EventType.EvtsJobExecution);
+
+            var events = _grid1.Events();
+
+            var eventFilter = new RemoteEventFilter(EventType.EvtJobStarted);
+
+            var oldEvents = events.RemoteQuery(eventFilter);
+
+            if (async)
+                events = events.WithAsync();
+
+            GenerateTaskEvent();
+
+            var remoteQuery = events.RemoteQuery(eventFilter, EventsTestHelper.Timeout, EventType.EvtsJobExecution);
+
+            if (async)
+            {
+                Assert.IsNull(remoteQuery);
+
+                remoteQuery = events.GetFuture<List<IEvent>>().Get().ToList();
+            }
+
+            var qryResult = remoteQuery.Except(oldEvents).Cast<JobEvent>().ToList();
+
+            Assert.AreEqual(_grids.Length, qryResult.Count);
+
+            Assert.IsTrue(qryResult.All(x => x.Type == EventType.EvtJobStarted));
+        }
+
+        /// <summary>
+        /// Tests serialization.
+        /// </summary>
+        [Test]
+        public void TestSerialization()
+        {
+            var grid = (Ignite) _grid1;
+            var comp = (Impl.Compute.Compute) grid.Cluster.ForLocal().Compute();
+            var locNode = grid.Cluster.LocalNode;
+
+            var expectedGuid = Guid.Parse("00000000-0000-0001-0000-000000000002");
+            var expectedGridGuid = new IgniteGuid(expectedGuid, 3);
+
+            using (var inStream = IgniteManager.Memory.Allocate().Stream())
+            {
+                var result = comp.ExecuteJavaTask<bool>("org.apache.ignite.platform.PlatformEventsWriteEventTask",
+                    inStream.MemoryPointer);
+
+                Assert.IsTrue(result);
+
+                inStream.SynchronizeInput();
+
+                var reader = grid.Marshaller.StartUnmarshal(inStream);
+
+                var cacheEvent = EventReader.Read<CacheEvent>(reader);
+                CheckEventBase(cacheEvent);
+                Assert.AreEqual("cacheName", cacheEvent.CacheName);
+                Assert.AreEqual(locNode, cacheEvent.EventNode);
+                Assert.AreEqual(1, cacheEvent.Partition);
+                Assert.AreEqual(true, cacheEvent.IsNear);
+                Assert.AreEqual(2, cacheEvent.Key);
+                Assert.AreEqual(expectedGridGuid, cacheEvent.Xid);
+                Assert.AreEqual(3, cacheEvent.LockId);
+                Assert.AreEqual(4, cacheEvent.NewValue);
+                Assert.AreEqual(true, cacheEvent.HasNewValue);
+                Assert.AreEqual(5, cacheEvent.OldValue);
+                Assert.AreEqual(true, cacheEvent.HasOldValue);
+                Assert.AreEqual(expectedGuid, cacheEvent.SubjectId);
+                Assert.AreEqual("cloClsName", cacheEvent.ClosureClassName);
+                Assert.AreEqual("taskName", cacheEvent.TaskName);
+
+                var qryExecEvent = EventReader.Read<CacheQueryExecutedEvent>(reader);
+                CheckEventBase(qryExecEvent);
+                Assert.AreEqual("qryType", qryExecEvent.QueryType);
+                Assert.AreEqual("cacheName", qryExecEvent.CacheName);
+                Assert.AreEqual("clsName", qryExecEvent.ClassName);
+                Assert.AreEqual("clause", qryExecEvent.Clause);
+                Assert.AreEqual(expectedGuid, qryExecEvent.SubjectId);
+                Assert.AreEqual("taskName", qryExecEvent.TaskName);
+
+                var qryReadEvent = EventReader.Read<CacheQueryReadEvent>(reader);
+                CheckEventBase(qryReadEvent);
+                Assert.AreEqual("qryType", qryReadEvent.QueryType);
+                Assert.AreEqual("cacheName", qryReadEvent.CacheName);
+                Assert.AreEqual("clsName", qryReadEvent.ClassName);
+                Assert.AreEqual("clause", qryReadEvent.Clause);
+                Assert.AreEqual(expectedGuid, qryReadEvent.SubjectId);
+                Assert.AreEqual("taskName", qryReadEvent.TaskName);
+                Assert.AreEqual(1, qryReadEvent.Key);
+                Assert.AreEqual(2, qryReadEvent.Value);
+                Assert.AreEqual(3, qryReadEvent.OldValue);
+                Assert.AreEqual(4, qryReadEvent.Row);
+
+                var cacheRebalancingEvent = EventReader.Read<CacheRebalancingEvent>(reader);
+                CheckEventBase(cacheRebalancingEvent);
+                Assert.AreEqual("cacheName", cacheRebalancingEvent.CacheName);
+                Assert.AreEqual(1, cacheRebalancingEvent.Partition);
+                Assert.AreEqual(locNode, cacheRebalancingEvent.DiscoveryNode);
+                Assert.AreEqual(2, cacheRebalancingEvent.DiscoveryEventType);
+                Assert.AreEqual(3, cacheRebalancingEvent.DiscoveryTimestamp);
+                
+                var checkpointEvent = EventReader.Read<CheckpointEvent>(reader);
+                CheckEventBase(checkpointEvent);
+                Assert.AreEqual("cpKey", checkpointEvent.Key);
+                
+                var discoEvent = EventReader.Read<DiscoveryEvent>(reader);
+                CheckEventBase(discoEvent);
+                Assert.AreEqual(grid.TopologyVersion, discoEvent.TopologyVersion);
+                Assert.AreEqual(grid.Nodes(), discoEvent.TopologyNodes);
+
+                var jobEvent = EventReader.Read<JobEvent>(reader);
+                CheckEventBase(jobEvent);
+                Assert.AreEqual(expectedGridGuid, jobEvent.JobId);
+                Assert.AreEqual("taskClsName", jobEvent.TaskClassName);
+                Assert.AreEqual("taskName", jobEvent.TaskName);
+                Assert.AreEqual(locNode, jobEvent.TaskNode);
+                Assert.AreEqual(expectedGridGuid, jobEvent.TaskSessionId);
+                Assert.AreEqual(expectedGuid, jobEvent.TaskSubjectId);
+
+                var spaceEvent = EventReader.Read<SwapSpaceEvent>(reader);
+                CheckEventBase(spaceEvent);
+                Assert.AreEqual("space", spaceEvent.Space);
+
+                var taskEvent = EventReader.Read<TaskEvent>(reader);
+                CheckEventBase(taskEvent);
+                Assert.AreEqual(true,taskEvent.Internal);
+                Assert.AreEqual(expectedGuid, taskEvent.SubjectId);
+                Assert.AreEqual("taskClsName", taskEvent.TaskClassName);
+                Assert.AreEqual("taskName", taskEvent.TaskName);
+                Assert.AreEqual(expectedGridGuid, taskEvent.TaskSessionId);
+            }
+        }
+
+        /// <summary>
+        /// Checks base event fields serialization.
+        /// </summary>
+        /// <param name="evt">The evt.</param>
+        private void CheckEventBase(IEvent evt)
+        {
+            var locNode = _grid1.Cluster.LocalNode;
+
+            Assert.AreEqual(locNode, evt.Node);
+            Assert.AreEqual("msg", evt.Message);
+            Assert.AreEqual(EventType.EvtSwapSpaceCleared, evt.Type);
+            Assert.IsNotNullOrEmpty(evt.Name);
+            Assert.AreNotEqual(Guid.Empty, evt.Id.GlobalId);
+            Assert.IsTrue((evt.TimeStamp - DateTime.Now).TotalSeconds < 10);
+        }
+
+        /// <summary>
+        /// Sends events in various ways and verifies correct receive.
+        /// </summary>
+        /// <param name="repeat">Expected event count multiplier.</param>
+        /// <param name="eventObjectType">Expected event object type.</param>
+        /// <param name="eventType">Type of the event.</param>
+        private void CheckSend(int repeat = 1, Type eventObjectType = null, params int[] eventType)
+        {
+            EventsTestHelper.ClearReceived(repeat);
+
+            GenerateTaskEvent();
+
+            EventsTestHelper.VerifyReceive(repeat, eventObjectType ?? typeof (TaskEvent),
+                eventType.Any() ? eventType : EventType.EvtsTaskExecution);
+        }
+
+        /// <summary>
+        /// Checks that no event has arrived.
+        /// </summary>
+        private void CheckNoEvent()
+        {
+            // this will result in an exception in case of a event
+            EventsTestHelper.ClearReceived(0);
+
+            GenerateTaskEvent();
+
+            Thread.Sleep(EventsTestHelper.Timeout);
+
+            EventsTestHelper.AssertFailures();
+        }
+
+        /// <summary>
+        /// Gets the Ignite configuration.
+        /// </summary>
+        private static IgniteConfiguration Configuration(string springConfigUrl)
+        {
+            return new IgniteConfiguration
+            {
+                SpringConfigUrl = springConfigUrl,
+                JvmClasspath = TestUtils.CreateTestClasspath(),
+                JvmOptions = TestUtils.TestJavaOptions(),
+                PortableConfiguration = new PortableConfiguration
+                {
+                    TypeConfigurations = new List<PortableTypeConfiguration>
+                    {
+                        new PortableTypeConfiguration(typeof (RemoteEventPortableFilter))
+                    }
+                }
+            };
+        }
+
+        /// <summary>
+        /// Generates the task event.
+        /// </summary>
+        private void GenerateTaskEvent(IIgnite grid = null)
+        {
+            (grid ?? _grid1).Compute().Broadcast(new ComputeAction());
+        }
+
+        /// <summary>
+        /// Verifies the task events.
+        /// </summary>
+        private static void VerifyTaskEvents(IEnumerable<IEvent> events)
+        {
+            var e = events.Cast<TaskEvent>().ToArray();
+
+            // started, reduced, finished
+            Assert.AreEqual(
+                new[] {EventType.EvtTaskStarted, EventType.EvtTaskReduced, EventType.EvtTaskFinished},
+                e.Select(x => x.Type).ToArray());
+        }
+
+        /// <summary>
+        /// Generates the cache query event.
+        /// </summary>
+        private static void GenerateCacheQueryEvent(IIgnite g)
+        {
+            var cache = g.Cache<int, int>(null);
+
+            cache.Clear();
+
+            cache.Put(1, 1);
+
+            cache.Query(new ScanQuery<int, int>()).GetAll();
+        }
+
+        /// <summary>
+        /// Verifies the cache events.
+        /// </summary>
+        private static void VerifyCacheEvents(IEnumerable<IEvent> events, IIgnite grid)
+        {
+            var e = events.Cast<CacheEvent>().ToArray();
+
+            foreach (var cacheEvent in e)
+            {
+                Assert.AreEqual(null, cacheEvent.CacheName);
+                Assert.AreEqual(null, cacheEvent.ClosureClassName);
+                Assert.AreEqual(null, cacheEvent.TaskName);
+                Assert.AreEqual(grid.Cluster.LocalNode, cacheEvent.EventNode);
+                Assert.AreEqual(grid.Cluster.LocalNode, cacheEvent.Node);
+
+                Assert.AreEqual(false, cacheEvent.HasOldValue);
+                Assert.AreEqual(null, cacheEvent.OldValue);
+
+                if (cacheEvent.Type == EventType.EvtCacheObjectPut)
+                {
+                    Assert.AreEqual(true, cacheEvent.HasNewValue);
+                    Assert.AreEqual(1, cacheEvent.NewValue);
+                }
+                else if (cacheEvent.Type == EventType.EvtCacheEntryCreated)
+                {
+                    Assert.AreEqual(false, cacheEvent.HasNewValue);
+                    Assert.AreEqual(null, cacheEvent.NewValue);
+                }
+                else
+                {
+                    Assert.Fail("Unexpected event type");
+                }
+            }
+        }
+
+        /// <summary>
+        /// Starts the grids.
+        /// </summary>
+        private void StartGrids()
+        {
+            if (_grid1 != null)
+                return;
+
+            _grid1 = Ignition.Start(Configuration("config\\compute\\compute-grid1.xml"));
+            _grid2 = Ignition.Start(Configuration("config\\compute\\compute-grid2.xml"));
+            _grid3 = Ignition.Start(Configuration("config\\compute\\compute-grid3.xml"));
+
+            _grids = new[] {_grid1, _grid2, _grid3};
+        }
+
+        /// <summary>
+        /// Stops the grids.
+        /// </summary>
+        private void StopGrids()
+        {
+            _grid1 = _grid2 = _grid3 = null;
+            _grids = null;
+            
+            Ignition.StopAll(true);
+        }
+    }
+
+    /// <summary>
+    /// Event test helper class.
+    /// </summary>
+    [Serializable]
+    public static class EventsTestHelper
+    {
+        /** */
+        public static readonly ConcurrentStack<IEvent> ReceivedEvents = new ConcurrentStack<IEvent>();
+        
+        /** */
+        public static readonly ConcurrentStack<string> Failures = new ConcurrentStack<string>();
+
+        /** */
+        public static readonly CountdownEvent ReceivedEvent = new CountdownEvent(0);
+
+        /** */
+        public static readonly ConcurrentStack<Guid> LastNodeIds = new ConcurrentStack<Guid>();
+
+        /** */
+        public static volatile bool ListenResult = true;
+
+        /** */
+        public static readonly TimeSpan Timeout = TimeSpan.FromMilliseconds(800);
+
+        /// <summary>
+        /// Clears received event information.
+        /// </summary>
+        /// <param name="expectedCount">The expected count of events to be received.</param>
+        public static void ClearReceived(int expectedCount)
+        {
+            ReceivedEvents.Clear();
+            ReceivedEvent.Reset(expectedCount);
+            LastNodeIds.Clear();
+        }
+
+        /// <summary>
+        /// Verifies received events against events events.
+        /// </summary>
+        public static void VerifyReceive(int count, Type eventObjectType, params int[] eventTypes)
+        {
+            // check if expected event count has been received; Wait returns false if there were none.
+            Assert.IsTrue(ReceivedEvent.Wait(Timeout), 
+                "Failed to receive expected number of events. Remaining count: " + ReceivedEvent.CurrentCount);
+            
+            Assert.AreEqual(count, ReceivedEvents.Count);
+
+            Assert.IsTrue(ReceivedEvents.All(x => x.GetType() == eventObjectType));
+
+            Assert.IsTrue(ReceivedEvents.All(x => eventTypes.Contains(x.Type)));
+
+            AssertFailures();
+        }
+
+        /// <summary>
+        /// Gets the event listener.
+        /// </summary>
+        /// <returns>New instance of event listener.</returns>
+        public static IEventFilter<IEvent> GetListener()
+        {
+            return new EventFilter<IEvent>(Listen);
+        }
+
+        /// <summary>
+        /// Combines accumulated failures and throws an assertion, if there are any.
+        /// Clears accumulated failures.
+        /// </summary>
+        public static void AssertFailures()
+        {
+            try
+            {
+                if (Failures.Any())
+                    Assert.Fail(Failures.Reverse().Aggregate((x, y) => string.Format("{0}\n{1}", x, y)));
+            }
+            finally 
+            {
+                Failures.Clear();
+            }
+        }
+
+        /// <summary>
+        /// Listen method.
+        /// </summary>
+        /// <param name="id">Originating node ID.</param>
+        /// <param name="evt">Event.</param>
+        private static bool Listen(Guid id, IEvent evt)
+        {
+            try
+            {
+                LastNodeIds.Push(id);
+                ReceivedEvents.Push(evt);
+
+                ReceivedEvent.Signal();
+                
+                return ListenResult;
+            }
+            catch (Exception ex)
+            {
+                // When executed on remote nodes, these exceptions will not go to sender, 
+                // so we have to accumulate them.
+                Failures.Push(string.Format("Exception in Listen (msg: {0}, id: {1}): {2}", evt, id, ex));
+                throw;
+            }
+        }
+    }
+
+    /// <summary>
+    /// Test event filter.
+    /// </summary>
+    [Serializable]
+    public class EventFilter<T> : IEventFilter<T> where T : IEvent
+    {
+        /** */
+        private readonly Func<Guid, T, bool> _invoke;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="RemoteListenEventFilter"/> class.
+        /// </summary>
+        /// <param name="invoke">The invoke delegate.</param>
+        public EventFilter(Func<Guid, T, bool> invoke)
+        {
+            _invoke = invoke;
+        }
+
+        /** <inheritdoc /> */
+        bool IEventFilter<T>.Invoke(Guid nodeId, T evt)
+        {
+            return _invoke(nodeId, evt);
+        }
+
+        /** <inheritdoc /> */
+        public bool Invoke(Guid nodeId, T evt)
+        {
+            throw new Exception("Invalid method");
+        }
+    }
+
+    /// <summary>
+    /// Remote event filter.
+    /// </summary>
+    [Serializable]
+    public class RemoteEventFilter : IEventFilter<IEvent>
+    {
+        /** */
+        private readonly int _type;
+
+        public RemoteEventFilter(int type)
+        {
+            _type = type;
+        }
+
+        /** <inheritdoc /> */
+        public bool Invoke(Guid nodeId, IEvent evt)
+        {
+            return evt.Type == _type;
+        }
+    }
+
+    /// <summary>
+    /// Portable remote event filter.
+    /// </summary>
+    public class RemoteEventPortableFilter : IEventFilter<IEvent>, IPortableMarshalAware
+    {
+        /** */
+        private int _type;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="RemoteEventPortableFilter"/> class.
+        /// </summary>
+        /// <param name="type">The event type.</param>
+        public RemoteEventPortableFilter(int type)
+        {
+            _type = type;
+        }
+
+        /** <inheritdoc /> */
+        public bool Invoke(Guid nodeId, IEvent evt)
+        {
+            return evt.Type == _type;
+        }
+
+        /** <inheritdoc /> */
+        public void WritePortable(IPortableWriter writer)
+        {
+            writer.RawWriter().WriteInt(_type);
+        }
+
+        /** <inheritdoc /> */
+        public void ReadPortable(IPortableReader reader)
+        {
+            _type = reader.RawReader().ReadInt();
+        }
+    }
+
+    /// <summary>
+    /// Event test case.
+    /// </summary>
+    public class EventTestCase
+    {
+        /// <summary>
+        /// Gets or sets the type of the event.
+        /// </summary>
+        public int[] EventType { get; set; }
+
+        /// <summary>
+        /// Gets or sets the type of the event object.
+        /// </summary>
+        public Type EventObjectType { get; set; }
+
+        /// <summary>
+        /// Gets or sets the generate event action.
+        /// </summary>
+        public Action<IIgnite> GenerateEvent { get; set; }
+
+        /// <summary>
+        /// Gets or sets the verify events action.
+        /// </summary>
+        public Action<IEnumerable<IEvent>, IIgnite> VerifyEvents { get; set; }
+
+        /// <summary>
+        /// Gets or sets the event count.
+        /// </summary>
+        public int EventCount { get; set; }
+
+        /** <inheritdoc /> */
+        public override string ToString()
+        {
+            return EventObjectType.ToString();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs
new file mode 100644
index 0000000..d90067f
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs
@@ -0,0 +1,352 @@
+/*
+ * 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.IO;
+    using System.Linq;
+    using System.Runtime.Serialization.Formatters.Binary;
+    using System.Threading.Tasks;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.Portable;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests grid exceptions propagation.
+    /// </summary>
+    public class ExceptionsTest
+    {
+        /// <summary>
+        /// Before test.
+        /// </summary>
+        [SetUp]
+        public void SetUp()
+        {
+            TestUtils.KillProcesses();
+        }
+        
+        /// <summary>
+        /// After test.
+        /// </summary>
+        [TearDown]
+        public void TearDown()
+        {
+            Ignition.StopAll(true);
+        }
+
+        /// <summary>
+        /// Tests exceptions.
+        /// </summary>
+        [Test]
+        public void TestExceptions()
+        {
+            var grid = StartGrid();
+
+            try
+            {
+                grid.Cache<object, object>("invalidCacheName");
+
+                Assert.Fail();
+            }
+            catch (Exception e)
+            {
+                Assert.IsTrue(e is ArgumentException);
+            }
+
+            try
+            {
+                grid.Cluster.ForRemotes().Metrics();
+
+                Assert.Fail();
+            }
+            catch (Exception e)
+            {
+                Assert.IsTrue(e is ClusterGroupEmptyException);
+            }
+
+            grid.Dispose();
+
+            try
+            {
+                grid.Cache<object, object>("cache1");
+
+                Assert.Fail();
+            }
+            catch (Exception e)
+            {
+                Assert.IsTrue(e is InvalidOperationException);
+            }
+        }
+
+        /// <summary>
+        /// Tests CachePartialUpdateException keys propagation.
+        /// </summary>
+        [Test]
+        [Category(TestUtils.CategoryIntensive)]
+        public void TestPartialUpdateException()
+        {
+            // Primitive type
+            TestPartialUpdateException(false, (x, g) => x);
+
+            // User type
+            TestPartialUpdateException(false, (x, g) => new PortableEntry(x));
+        }
+
+        /// <summary>
+        /// Tests CachePartialUpdateException keys propagation in portable mode.
+        /// </summary>
+        [Test]
+        [Category(TestUtils.CategoryIntensive)]
+        public void TestPartialUpdateExceptionPortable()
+        {
+            // User type
+            TestPartialUpdateException(false, (x, g) => g.Portables().ToPortable<IPortableObject>(new PortableEntry(x)));
+        }
+
+        /// <summary>
+        /// Tests CachePartialUpdateException serialization.
+        /// </summary>
+        [Test]
+        public void TestPartialUpdateExceptionSerialization()
+        {
+            // Inner exception
+            TestPartialUpdateExceptionSerialization(new CachePartialUpdateException("Msg",
+                new IgniteException("Inner msg")));
+
+            // Primitive keys
+            TestPartialUpdateExceptionSerialization(new CachePartialUpdateException("Msg", new object[] {1, 2, 3}));
+
+            // User type keys
+            TestPartialUpdateExceptionSerialization(new CachePartialUpdateException("Msg",
+                new object[]
+                {
+                    new SerializableEntry(1), 
+                    new SerializableEntry(2),
+                    new SerializableEntry(3)
+                }));
+        }
+
+        /// <summary>
+        /// Tests CachePartialUpdateException serialization.
+        /// </summary>
+        private static void TestPartialUpdateExceptionSerialization(Exception ex)
+        {
+            var formatter = new BinaryFormatter();
+
+            var stream = new MemoryStream();
+
+            formatter.Serialize(stream, ex);
+
+            stream.Seek(0, SeekOrigin.Begin);
+
+            var ex0 = (Exception) formatter.Deserialize(stream);
+                
+            var updateEx = ((CachePartialUpdateException) ex);
+
+            try
+            {
+                Assert.AreEqual(updateEx.GetFailedKeys<object>(),
+                    ((CachePartialUpdateException)ex0).GetFailedKeys<object>());
+            }
+            catch (Exception e)
+            {
+                if (typeof (IgniteException) != e.GetType())
+                    throw;
+            }
+
+            while (ex != null && ex0 != null)
+            {
+                Assert.AreEqual(ex0.GetType(), ex.GetType());
+                Assert.AreEqual(ex.Message, ex0.Message);
+
+                ex = ex.InnerException;
+                ex0 = ex0.InnerException;
+            }
+
+            Assert.AreEqual(ex, ex0);
+        }
+
+        /// <summary>
+        /// Tests CachePartialUpdateException keys propagation.
+        /// </summary>
+        [Test]
+        [Category(TestUtils.CategoryIntensive)]
+        public void TestPartialUpdateExceptionAsync()
+        {
+            // Primitive type
+            TestPartialUpdateException(true, (x, g) => x);
+
+            // User type
+            TestPartialUpdateException(true, (x, g) => new PortableEntry(x));
+        }
+
+        /// <summary>
+        /// Tests CachePartialUpdateException keys propagation in portable mode.
+        /// </summary>
+        [Test]
+        [Category(TestUtils.CategoryIntensive)]
+        public void TestPartialUpdateExceptionAsyncPortable()
+        {
+            TestPartialUpdateException(true, (x, g) => g.Portables().ToPortable<IPortableObject>(new PortableEntry(x)));
+        }
+
+        /// <summary>
+        /// Tests CachePartialUpdateException keys propagation.
+        /// </summary>
+        private static void TestPartialUpdateException<TK>(bool async, Func<int, IIgnite, TK> keyFunc)
+        {
+            using (var grid = StartGrid())
+            {
+                var cache = grid.Cache<TK, int>("partitioned_atomic").WithNoRetries();
+
+                if (async)
+                    cache = cache.WithAsync();
+
+                if (typeof (TK) == typeof (IPortableObject))
+                    cache = cache.WithKeepPortable<TK, int>();
+
+                // Do cache puts in parallel
+                var putTask = Task.Factory.StartNew(() =>
+                {
+                    try
+                    {
+                        // Do a lot of puts so that one fails during Ignite stop
+                        for (var i = 0; i < 1000000; i++)
+                        {
+                            cache.PutAll(Enumerable.Range(1, 100).ToDictionary(k => keyFunc(k, grid), k => i));
+
+                            if (async)
+                                cache.GetFuture().Get();
+                        }
+                    }
+                    catch (CachePartialUpdateException ex)
+                    {
+                        var failedKeys = ex.GetFailedKeys<TK>();
+
+                        Assert.IsTrue(failedKeys.Any());
+
+                        var failedKeysObj = ex.GetFailedKeys<object>();
+
+                        Assert.IsTrue(failedKeysObj.Any());
+
+                        return;
+                    }
+
+                    Assert.Fail("CachePartialUpdateException has not been thrown.");
+                });
+
+                while (true)
+                {
+                    Ignition.Stop("grid_2", true);
+                    StartGrid("grid_2");
+
+                    if (putTask.Exception != null)
+                        throw putTask.Exception;
+
+                    if (putTask.IsCompleted)
+                        return;
+                }
+            }
+        }
+
+        /// <summary>
+        /// Starts the grid.
+        /// </summary>
+        private static IIgnite StartGrid(string gridName = null)
+        {
+            return Ignition.Start(new IgniteConfigurationEx
+            {
+                SpringConfigUrl = "config\\native-client-test-cache.xml",
+                JvmOptions = TestUtils.TestJavaOptions(),
+                JvmClasspath = TestUtils.CreateTestClasspath(),
+                GridName = gridName,
+                PortableConfiguration = new PortableConfiguration
+                {
+                    TypeConfigurations = new[]
+                    {
+                        new PortableTypeConfiguration(typeof (PortableEntry))
+                    }
+                }
+            });
+        }
+
+        /// <summary>
+        /// Portable entry.
+        /// </summary>
+        private class PortableEntry
+        {
+            /** Value. */
+            private readonly int _val;
+
+            /** <inheritDot /> */
+            public override int GetHashCode()
+            {
+                return _val;
+            }
+
+            /// <summary>
+            /// Constructor.
+            /// </summary>
+            /// <param name="val">Value.</param>
+            public PortableEntry(int val)
+            {
+                _val = val;
+            }
+
+            /** <inheritDoc /> */
+            public override bool Equals(object obj)
+            {
+                return obj is PortableEntry && ((PortableEntry)obj)._val == _val;
+            }
+        }
+
+        /// <summary>
+        /// Portable entry.
+        /// </summary>
+        [Serializable]
+        private class SerializableEntry
+        {
+            /** Value. */
+            private readonly int _val;
+
+            /** <inheritDot /> */
+            public override int GetHashCode()
+            {
+                return _val;
+            }
+
+            /// <summary>
+            /// Constructor.
+            /// </summary>
+            /// <param name="val">Value.</param>
+            public SerializableEntry(int val)
+            {
+                _val = val;
+            }
+
+            /** <inheritDoc /> */
+            public override bool Equals(object obj)
+            {
+                return obj is SerializableEntry && ((SerializableEntry)obj)._val == _val;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/ExecutableTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/ExecutableTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/ExecutableTest.cs
new file mode 100644
index 0000000..9c47cbc
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/ExecutableTest.cs
@@ -0,0 +1,444 @@
+/*
+ * 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 UnusedVariable
+// ReSharper disable UnusedAutoPropertyAccessor.Global
+namespace Apache.Ignite.Core.Tests
+{
+    using System;
+    using System.CodeDom.Compiler;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.Portable;
+    using Apache.Ignite.Core.Resource;
+    using Apache.Ignite.Core.Tests.Process;
+    using Microsoft.CSharp;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests for executable.
+    /// </summary>
+    [Ignore("IGNITE-1367")]
+    public class ExecutableTest
+    {
+        /** Spring configuration path. */
+        private static readonly string SpringCfgPath = "config\\compute\\compute-standalone.xml";
+
+        /** Min memory Java task. */
+        private const string MinMemTask = "org.apache.ignite.platform.PlatformMinMemoryTask";
+
+        /** Max memory Java task. */
+        private const string MaxMemTask = "org.apache.ignite.platform.PlatformMaxMemoryTask";
+
+        /** Grid. */
+        private IIgnite _grid;
+
+        /// <summary>
+        /// Test fixture set-up routine.
+        /// </summary>
+        [TestFixtureSetUp]
+        public void TestFixtureSetUp()
+        {
+            TestUtils.KillProcesses();
+
+            _grid = Ignition.Start(Configuration(SpringCfgPath));
+        }
+
+        /// <summary>
+        /// Test fixture tear-down routine.
+        /// </summary>
+        [TestFixtureTearDown]
+        public void TestFixtureTearDown()
+        {
+            Ignition.StopAll(true);
+
+            TestUtils.KillProcesses();
+        }
+
+        /// <summary>
+        /// Set-up routine.
+        /// </summary>
+        [SetUp]
+        public void SetUp()
+        {
+            TestUtils.KillProcesses();
+
+            Assert.IsTrue(_grid.WaitTopology(1, 30000));
+
+            IgniteProcess.SaveConfigurationBackup();
+        }
+
+        /// <summary>
+        /// Tear-down routine.
+        /// </summary>
+        [TearDown]
+        public void TearDown()
+        {
+            IgniteProcess.RestoreConfigurationBackup();
+        }
+
+        /// <summary>
+        /// Test data pass through configuration file.
+        /// </summary>
+        [Test]
+        public void TestConfig()
+        {
+            IgniteProcess.ReplaceConfiguration("config\\Ignite.exe.config.test");
+
+            GenerateDll("test-1.dll");
+            GenerateDll("test-2.dll");
+
+            IgniteProcess proc = new IgniteProcess(
+                "-jvmClasspath=" + TestUtils.CreateTestClasspath()
+                );
+
+            Assert.IsTrue(_grid.WaitTopology(2, 30000));
+
+            RemoteConfiguration cfg = RemoteConfig();
+
+            Assert.AreEqual(SpringCfgPath, cfg.SpringConfigUrl);
+            Assert.IsTrue(cfg.JvmOptions.Contains("-DOPT1") && cfg.JvmOptions.Contains("-DOPT2"));
+            Assert.IsTrue(cfg.Assemblies.Contains("test-1.dll") && cfg.Assemblies.Contains("test-2.dll"));
+            Assert.AreEqual(601, cfg.JvmInitialMemoryMb);
+            Assert.AreEqual(702, cfg.JvmMaxMemoryMb);
+        }
+
+        /// <summary>
+        /// Test assemblies passing through command-line. 
+        /// </summary>
+        [Test]
+        public void TestAssemblyCmd()
+        {
+            GenerateDll("test-1.dll");
+            GenerateDll("test-2.dll");
+
+            IgniteProcess proc = new IgniteProcess(
+                "-jvmClasspath=" + TestUtils.CreateTestClasspath(),
+                "-springConfigUrl=" + SpringCfgPath,
+                "-assembly=test-1.dll",
+                "-assembly=test-2.dll"
+                );
+
+            Assert.IsTrue(_grid.WaitTopology(2, 30000));
+
+            RemoteConfiguration cfg = RemoteConfig();
+
+            Assert.IsTrue(cfg.Assemblies.Contains("test-1.dll") && cfg.Assemblies.Contains("test-2.dll"));
+        }
+
+        /// <summary>
+        /// Test JVM options passing through command-line. 
+        /// </summary>
+        [Test]
+        public void TestJvmOptsCmd()
+        {
+            IgniteProcess proc = new IgniteProcess(
+                "-jvmClasspath=" + TestUtils.CreateTestClasspath(),
+                "-springConfigUrl=" + SpringCfgPath,
+                "-J-DOPT1",
+                "-J-DOPT2"
+                );
+
+            Assert.IsTrue(_grid.WaitTopology(2, 30000));
+
+            RemoteConfiguration cfg = RemoteConfig();
+
+            Assert.IsTrue(cfg.JvmOptions.Contains("-DOPT1") && cfg.JvmOptions.Contains("-DOPT2"));
+        }
+
+        /// <summary>
+        /// Test JVM memory options passing through command-line: raw java options.
+        /// </summary>
+        [Test]
+        public void TestJvmMemoryOptsCmdRaw()
+        {
+            var proc = new IgniteProcess(
+                "-jvmClasspath=" + TestUtils.CreateTestClasspath(),
+                "-springConfigUrl=" + SpringCfgPath,
+                "-J-Xms506m",
+                "-J-Xmx607m"
+                );
+
+            Assert.IsTrue(_grid.WaitTopology(2, 30000));
+
+            var minMem = _grid.Cluster.ForRemotes().Compute().ExecuteJavaTask<long>(MinMemTask, null);
+            Assert.AreEqual((long) 506*1024*1024, minMem);
+
+            var maxMem = _grid.Cluster.ForRemotes().Compute().ExecuteJavaTask<long>(MaxMemTask, null);
+            AssertJvmMaxMemory((long) 607*1024*1024, maxMem);
+        }
+
+        /// <summary>
+        /// Test JVM memory options passing through command-line: custom options.
+        /// </summary>
+        [Test]
+        public void TestJvmMemoryOptsCmdCustom()
+        {
+            var proc = new IgniteProcess(
+                "-jvmClasspath=" + TestUtils.CreateTestClasspath(),
+                "-springConfigUrl=" + SpringCfgPath,
+                "-JvmInitialMemoryMB=615",
+                "-JvmMaxMemoryMB=863"
+                );
+
+            Assert.IsTrue(_grid.WaitTopology(2, 30000));
+
+            var minMem = _grid.Cluster.ForRemotes().Compute().ExecuteJavaTask<long>(MinMemTask, null);
+            Assert.AreEqual((long) 615*1024*1024, minMem);
+
+            var maxMem = _grid.Cluster.ForRemotes().Compute().ExecuteJavaTask<long>(MaxMemTask, null);
+            AssertJvmMaxMemory((long) 863*1024*1024, maxMem);
+        }
+
+        /// <summary>
+        /// Test JVM memory options passing from application configuration.
+        /// </summary>
+        [Test]
+        public void TestJvmMemoryOptsAppConfig()
+        {
+            IgniteProcess.ReplaceConfiguration("config\\Ignite.exe.config.test");
+
+            GenerateDll("test-1.dll");
+            GenerateDll("test-2.dll");
+
+            var proc = new IgniteProcess("-jvmClasspath=" + TestUtils.CreateTestClasspath());
+
+            Assert.IsTrue(_grid.WaitTopology(2, 30000));
+
+            var minMem = _grid.Cluster.ForRemotes().Compute().ExecuteJavaTask<long>(MinMemTask, null);
+            Assert.AreEqual((long) 601*1024*1024, minMem);
+
+            var maxMem = _grid.Cluster.ForRemotes().Compute().ExecuteJavaTask<long>(MaxMemTask, null);
+            AssertJvmMaxMemory((long) 702*1024*1024, maxMem);
+
+            proc.Kill();
+
+            Assert.IsTrue(_grid.WaitTopology(1, 30000));
+
+            // Command line options overwrite config file options
+            // ReSharper disable once RedundantAssignment
+            proc = new IgniteProcess("-jvmClasspath=" + TestUtils.CreateTestClasspath(),
+                "-J-Xms605m", "-J-Xmx706m");
+
+            Assert.IsTrue(_grid.WaitTopology(2, 30000));
+
+            minMem = _grid.Cluster.ForRemotes().Compute().ExecuteJavaTask<long>(MinMemTask, null);
+            Assert.AreEqual((long) 605*1024*1024, minMem);
+
+            maxMem = _grid.Cluster.ForRemotes().Compute().ExecuteJavaTask<long>(MaxMemTask, null);
+            AssertJvmMaxMemory((long) 706*1024*1024, maxMem);
+        }
+
+        /// <summary>
+        /// Test JVM memory options passing through command-line: custom options + raw options.
+        /// </summary>
+        [Test]
+        public void TestJvmMemoryOptsCmdCombined()
+        {
+            var proc = new IgniteProcess(
+                "-jvmClasspath=" + TestUtils.CreateTestClasspath(),
+                "-springConfigUrl=" + SpringCfgPath,
+                "-J-Xms555m",
+                "-J-Xmx666m",
+                "-JvmInitialMemoryMB=128",
+                "-JvmMaxMemoryMB=256"
+                );
+
+            Assert.IsTrue(_grid.WaitTopology(2, 30000));
+
+            // Raw JVM options (Xms/Xmx) should override custom options
+            var minMem = _grid.Cluster.ForRemotes().Compute().ExecuteJavaTask<long>(MinMemTask, null);
+            Assert.AreEqual((long) 555*1024*1024, minMem);
+
+            var maxMem = _grid.Cluster.ForRemotes().Compute().ExecuteJavaTask<long>(MaxMemTask, null);
+            AssertJvmMaxMemory((long) 666*1024*1024, maxMem);
+        }
+
+        /// <summary>
+        /// Get remote node configuration.
+        /// </summary>
+        /// <returns>Configuration.</returns>
+        private RemoteConfiguration RemoteConfig()
+        {
+            return _grid.Cluster.ForRemotes().Compute().Call(new RemoteConfigurationClosure());
+        }
+
+        /// <summary>
+        /// Configuration for node.
+        /// </summary>
+        /// <param name="path">Path to Java XML configuration.</param>
+        /// <returns>Node configuration.</returns>
+        private static IgniteConfiguration Configuration(string path)
+        {
+            IgniteConfiguration cfg = new IgniteConfiguration();
+
+
+            PortableConfiguration portCfg = new PortableConfiguration();
+
+            ICollection<PortableTypeConfiguration> portTypeCfgs = new List<PortableTypeConfiguration>();
+
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof (RemoteConfiguration)));
+            portTypeCfgs.Add(new PortableTypeConfiguration(typeof (RemoteConfigurationClosure)));
+
+            portCfg.TypeConfigurations = portTypeCfgs;
+
+            cfg.PortableConfiguration = portCfg;
+
+            cfg.JvmClasspath = TestUtils.CreateTestClasspath();
+
+            cfg.JvmOptions = new List<string>
+            {
+                "-ea",
+                "-Xcheck:jni",
+                "-Xms4g",
+                "-Xmx4g",
+                "-DGRIDGAIN_QUIET=false",
+                "-Xnoagent",
+                "-Djava.compiler=NONE",
+                "-Xdebug",
+                "-Xrunjdwp:transport=dt_socket,server=y,suspend=n,address=5005",
+                "-XX:+HeapDumpOnOutOfMemoryError"
+            };
+
+            cfg.SpringConfigUrl = path;
+
+            return cfg;
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        /// <param name="outputPath"></param>
+        private static void GenerateDll(string outputPath)
+        {
+            CSharpCodeProvider codeProvider = new CSharpCodeProvider();
+
+#pragma warning disable 0618
+
+            ICodeCompiler icc = codeProvider.CreateCompiler();
+
+#pragma warning restore 0618
+
+            CompilerParameters parameters = new CompilerParameters();
+            parameters.GenerateExecutable = false;
+            parameters.OutputAssembly = outputPath;
+
+            string src = "namespace GridGain.Client.Test { public class Foo {}}";
+
+            CompilerResults results = icc.CompileAssemblyFromSource(parameters, src);
+
+            Assert.False(results.Errors.HasErrors);
+        }
+
+        /// <summary>
+        /// Asserts that JVM maximum memory corresponds to Xmx parameter value.
+        /// </summary>
+        private static void AssertJvmMaxMemory(long expected, long actual)
+        {
+            // allow 20% tolerance because max memory in Java is not exactly equal to Xmx parameter value
+            Assert.LessOrEqual(actual, expected);
+            Assert.Greater(actual, expected/5*4);
+        }
+
+        /// <summary>
+        /// Closure which extracts configuration and passes it back.
+        /// </summary>
+        public class RemoteConfigurationClosure : IComputeFunc<RemoteConfiguration>
+        {
+
+#pragma warning disable 0649
+
+            /** Grid. */
+            [InstanceResource] private IIgnite _grid;
+
+#pragma warning restore 0649
+
+            /** <inheritDoc /> */
+
+            public RemoteConfiguration Invoke()
+            {
+                Ignite grid0 = (Ignite) ((IgniteProxy) _grid).Target;
+
+                IgniteConfiguration cfg = grid0.Configuration;
+
+                RemoteConfiguration res = new RemoteConfiguration
+                {
+                    IgniteHome = cfg.IgniteHome,
+                    SpringConfigUrl = cfg.SpringConfigUrl,
+                    JvmDll = cfg.JvmDllPath,
+                    JvmClasspath = cfg.JvmClasspath,
+                    JvmOptions = cfg.JvmOptions,
+                    Assemblies = cfg.Assemblies,
+                    JvmInitialMemoryMb = cfg.JvmInitialMemoryMb,
+                    JvmMaxMemoryMb = cfg.JvmMaxMemoryMb
+                };
+
+                Console.WriteLine("RETURNING CFG: " + cfg);
+
+                return res;
+            }
+        }
+
+        /// <summary>
+        /// Configuration.
+        /// </summary>
+        public class RemoteConfiguration
+        {
+            /// <summary>
+            /// GG home.
+            /// </summary>
+            public string IgniteHome { get; set; }
+
+            /// <summary>
+            /// Spring config URL.
+            /// </summary>
+            public string SpringConfigUrl { get; set; }
+
+            /// <summary>
+            /// JVM DLL.
+            /// </summary>
+            public string JvmDll { get; set; }
+
+            /// <summary>
+            /// JVM classpath.
+            /// </summary>
+            public string JvmClasspath { get; set; }
+
+            /// <summary>
+            /// JVM options.
+            /// </summary>
+            public ICollection<string> JvmOptions { get; set; }
+
+            /// <summary>
+            /// Assemblies.
+            /// </summary>
+            public ICollection<string> Assemblies { get; set; }
+
+            /// <summary>
+            /// Minimum JVM memory (Xms).
+            /// </summary>
+            public int JvmInitialMemoryMb { get; set; }
+
+            /// <summary>
+            /// Maximum JVM memory (Xms).
+            /// </summary>
+            public int JvmMaxMemoryMb { get; set; }
+
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/FutureTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/FutureTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/FutureTest.cs
new file mode 100644
index 0000000..c2815db
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/FutureTest.cs
@@ -0,0 +1,278 @@
+/*
+ * 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.Collections.Generic;
+    using System.Threading;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Portable;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Future tests.
+    /// </summary>
+    public class FutureTest
+    {
+        /** */
+        private ICache<object, object> _cache;
+
+        /** */
+        private ICompute _compute;
+
+        /// <summary>
+        /// Test fixture set-up routine.
+        /// </summary>
+        [TestFixtureSetUp]
+        public void TestFixtureSetUp()
+        {
+            TestUtils.KillProcesses();
+
+            var grid = Ignition.Start(new IgniteConfiguration
+            {
+                SpringConfigUrl = "config\\compute\\compute-standalone.xml",
+                JvmClasspath = TestUtils.CreateTestClasspath(),
+                JvmOptions = TestUtils.TestJavaOptions(),
+                PortableConfiguration = new PortableConfiguration
+                {
+                    TypeConfigurations =
+                        new List<PortableTypeConfiguration> { new PortableTypeConfiguration(typeof(Portable)) }
+                }
+            });
+
+            _cache = grid.Cache<object, object>(null).WithAsync();
+
+            _compute = grid.Compute().WithAsync();
+        }
+
+        /// <summary>
+        /// Test fixture tear-down routine.
+        /// </summary>
+        [TestFixtureTearDown]
+        public void TestFixtureTearDown()
+        {
+            TestUtils.KillProcesses();
+        }
+
+        [Test]
+        public void TestListen()
+        {
+            // Listen(Action callback)
+            TestListen((fut, act) => fut.Listen(act));
+
+            // Listen(Action<IFuture> callback)
+            TestListen((fut, act) => ((IFuture)fut).Listen(f =>
+            {
+                Assert.AreEqual(f, fut);
+                act();
+            }));
+
+            // Listen(Action<IFuture<T>> callback)
+            TestListen((fut, act) => fut.Listen(f =>
+            {
+                Assert.AreEqual(f, fut);
+                act();
+            }));
+        }
+
+        private void TestListen(Action<IFuture<object>, Action> listenAction)
+        {
+            _compute.Broadcast(new SleepAction());
+
+            var fut = _compute.GetFuture<object>();
+
+            var listenCount = 0;
+
+            // Multiple subscribers before completion
+            for (var i = 0; i < 10; i++)
+                listenAction(fut, () => Interlocked.Increment(ref listenCount));
+
+            Assert.IsFalse(fut.IsDone);
+
+            Assert.IsNull(fut.Get());
+
+            Thread.Sleep(100);  // wait for future completion thread
+
+            Assert.AreEqual(10, listenCount);
+
+            // Multiple subscribers after completion
+            for (var i = 0; i < 10; i++)
+                listenAction(fut, () => Interlocked.Decrement(ref listenCount));
+
+            Assert.AreEqual(0, listenCount);
+        }
+
+        [Test]
+        public void TestToTask()
+        {
+            _cache.Put(1, 1);
+
+            _cache.GetFuture().ToTask().Wait();
+
+            _cache.Get(1);
+
+            var task1 = _cache.GetFuture<int>().ToTask();
+
+            Assert.AreEqual(1, task1.Result);
+
+            Assert.IsTrue(task1.IsCompleted);
+
+            _compute.Broadcast(new SleepAction());
+
+            var task2 = _compute.GetFuture().ToTask();
+
+            Assert.IsFalse(task2.IsCompleted);
+
+            Assert.IsFalse(task2.Wait(100));
+
+            task2.Wait();
+
+            Assert.IsTrue(task2.IsCompleted);
+
+            Assert.AreEqual(null, task2.Result);
+        }
+
+        [Test]
+        public void TestGetWithTimeout()
+        {
+            _compute.Broadcast(new SleepAction());
+
+            var fut = _compute.GetFuture();
+
+            Assert.Throws<TimeoutException>(() => fut.Get(TimeSpan.FromMilliseconds(100)));
+
+            fut.Get(TimeSpan.FromSeconds(1));
+
+            Assert.IsTrue(fut.IsDone);
+        }
+
+        [Test]
+        public void TestToAsyncResult()
+        {
+            _compute.Broadcast(new SleepAction());
+
+            IFuture fut = _compute.GetFuture();
+
+            var asyncRes = fut.ToAsyncResult();
+
+            Assert.IsFalse(asyncRes.IsCompleted);
+
+            Assert.IsTrue(asyncRes.AsyncWaitHandle.WaitOne(1000));
+
+            Assert.IsTrue(asyncRes.IsCompleted);
+        }
+
+        [Test]
+        public void TestFutureTypes()
+        {
+            TestType(false);
+            TestType((byte)11);
+            TestType('x'); // char
+            TestType(2.7d); // double
+            TestType(3.14f); // float
+            TestType(16); // int
+            TestType(17L); // long
+            TestType((short)18);
+
+            TestType(18m); // decimal
+
+            TestType(new Portable { A = 10, B = "foo" });
+        }
+
+        /// <summary>
+        /// Tests future type.
+        /// </summary>
+        private void TestType<T>(T value)
+        {
+            var key = typeof(T).Name;
+
+            _cache.Put(key, value);
+
+            _cache.GetFuture().Get();
+
+            _cache.Get(key);
+
+            Assert.AreEqual(value, _cache.GetFuture<T>().Get());
+        }
+
+        /// <summary>
+        /// Portable test class.
+        /// </summary>
+        private class Portable : IPortableMarshalAware
+        {
+            public int A;
+            public string B;
+
+            /** <inheritDoc /> */
+            public void WritePortable(IPortableWriter writer)
+            {
+                writer.WriteInt("a", A);
+                writer.RawWriter().WriteString(B);
+            }
+
+            /** <inheritDoc /> */
+            public void ReadPortable(IPortableReader reader)
+            {
+                A = reader.ReadInt("a");
+                B = reader.RawReader().ReadString();
+            }
+
+            /** <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;
+
+                var other = (Portable)obj;
+
+                return A == other.A && string.Equals(B, other.B);
+            }
+
+            /** <inheritDoc /> */
+            public override int GetHashCode()
+            {
+                unchecked
+                {
+                    // ReSharper disable NonReadonlyMemberInGetHashCode
+                    return (A * 397) ^ (B != null ? B.GetHashCode() : 0);
+                    // ReSharper restore NonReadonlyMemberInGetHashCode
+                }
+            }
+        }
+
+        /// <summary>
+        /// Compute action with a delay to ensure lengthy future execution.
+        /// </summary>
+        [Serializable]
+        private class SleepAction : IComputeAction
+        {
+            public void Invoke()
+            {
+                Thread.Sleep(500);
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/IgniteManagerTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/IgniteManagerTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/IgniteManagerTest.cs
new file mode 100644
index 0000000..5a90c20
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/IgniteManagerTest.cs
@@ -0,0 +1,51 @@
+/*
+ * 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.IO;
+    using Apache.Ignite.Core.Impl;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests IgniteManager class.
+    /// </summary>
+    public class IgniteManagerTest
+    {
+        /// <summary>
+        /// Tests home dir resolver.
+        /// </summary>
+        [Test]
+        public void TestIgniteHome()
+        {
+            var env = Environment.GetEnvironmentVariable(IgniteManager.EnvIgniteHome);
+            
+            Environment.SetEnvironmentVariable(IgniteManager.EnvIgniteHome, null);
+
+            try
+            {
+                Assert.IsTrue(Directory.Exists(IgniteManager.GetIgniteHome(null)));
+            }
+            finally
+            {
+                // Restore
+                Environment.SetEnvironmentVariable(IgniteManager.EnvIgniteHome, env);
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/IgniteStartStopTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/IgniteStartStopTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/IgniteStartStopTest.cs
new file mode 100644
index 0000000..d2b2efa
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/IgniteStartStopTest.cs
@@ -0,0 +1,380 @@
+/*
+ * 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.Collections.Generic;
+    using System.Threading;
+    using Apache.Ignite.Core.Common;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Ignite start/stop tests.
+    /// </summary>
+    [Category(TestUtils.CategoryIntensive)]
+    public class IgniteStartStopTest
+    {
+        /// <summary>
+        /// 
+        /// </summary>
+        [SetUp]
+        public void SetUp()
+        {
+            TestUtils.KillProcesses();
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        [TearDown]
+        public void TearDown()
+        {
+            Ignition.StopAll(true);
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        [Test]
+        public void TestStartDefault()
+        {
+            var cfg = new IgniteConfiguration {JvmClasspath = TestUtils.CreateTestClasspath()};
+
+            var grid = Ignition.Start(cfg);
+
+            Assert.IsNotNull(grid);
+
+            Assert.AreEqual(1, grid.Cluster.Nodes().Count);
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        [Test]
+        public void TestStartWithConfigPath()
+        {
+            var cfg = new IgniteConfiguration
+            {
+                SpringConfigUrl = "config/default-config.xml",
+                JvmClasspath = TestUtils.CreateTestClasspath()
+            };
+
+            var grid = Ignition.Start(cfg);
+
+            Assert.IsNotNull(grid);
+
+            Assert.AreEqual(1, grid.Cluster.Nodes().Count);
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        [Test]
+        public void TestStartGetStop()
+        {
+            var cfgs = new List<string> { "config\\start-test-grid1.xml", "config\\start-test-grid2.xml", "config\\start-test-grid3.xml" };
+
+            var cfg = new IgniteConfiguration
+            {
+                SpringConfigUrl = cfgs[0],
+                JvmOptions = TestUtils.TestJavaOptions(),
+                JvmClasspath = TestUtils.CreateTestClasspath()
+            };
+
+            var grid1 = Ignition.Start(cfg);
+
+            Assert.AreEqual("grid1", grid1.Name);
+
+            cfg.SpringConfigUrl = cfgs[1];
+
+            var grid2 = Ignition.Start(cfg);
+
+            Assert.AreEqual("grid2", grid2.Name);
+
+            cfg.SpringConfigUrl = cfgs[2];
+
+            var grid3 = Ignition.Start(cfg);
+
+            Assert.IsNull(grid3.Name);
+
+            Assert.AreSame(grid1, Ignition.GetIgnite("grid1"));
+
+            Assert.AreSame(grid2, Ignition.GetIgnite("grid2"));
+
+            Assert.AreSame(grid3, Ignition.GetIgnite(null));
+
+            try
+            {
+                Ignition.GetIgnite("invalid_name");
+            }
+            catch (IgniteException e)
+            {
+                Console.WriteLine("Expected exception: " + e);
+            }
+
+            Assert.IsTrue(Ignition.Stop("grid1", true));
+
+            try
+            {
+                Ignition.GetIgnite("grid1");
+            }
+            catch (IgniteException e)
+            {
+                Console.WriteLine("Expected exception: " + e);
+            }
+
+            grid2.Dispose();
+
+            try
+            {
+                Ignition.GetIgnite("grid2");
+            }
+            catch (IgniteException e)
+            {
+                Console.WriteLine("Expected exception: " + e);
+            }
+
+            grid3.Dispose();
+
+            try
+            {
+                Ignition.GetIgnite(null);
+            }
+            catch (IgniteException e)
+            {
+                Console.WriteLine("Expected exception: " + e);
+            }
+
+            foreach (var cfgName in cfgs)
+            {
+                cfg.SpringConfigUrl = cfgName;
+                cfg.JvmOptions = TestUtils.TestJavaOptions();
+
+                Ignition.Start(cfg);
+            }
+
+            foreach (var gridName in new List<string> { "grid1", "grid2", null })
+                Assert.IsNotNull(Ignition.GetIgnite(gridName));
+
+            Ignition.StopAll(true);
+
+            foreach (var gridName in new List<string> { "grid1", "grid2", null })
+            {
+                try
+                {
+                    Ignition.GetIgnite(gridName);
+                }
+                catch (IgniteException e)
+                {
+                    Console.WriteLine("Expected exception: " + e);
+                }
+            }
+        }
+
+        /*
+        [Test]
+        public void TestStartInvalidJvmOptions()
+        {
+            GridGain.Impl.IgniteManager.DestroyJvm();
+
+            IgniteConfiguration cfg = new IgniteConfiguration();
+
+            cfg.NativeXmlConfig = "config\\start-test-grid1.xml";
+            cfg.NativeJvmOptions = new List<string> { "invalid_option"};
+
+            try
+            {
+                Ignition.Start(cfg);
+
+                Assert.Fail("Start should fail.");
+            }
+            catch (IgniteException e)
+            {
+                Console.WriteLine("Expected exception: " + e);
+            }
+
+            cfg.NativeJvmOptions = new List<string> { "-Xmx1g", "-Xms1g" };
+
+            Ignition.Start(cfg);
+        }
+        */
+
+        /// <summary>
+        /// 
+        /// </summary>
+        [Test]
+        public void TestStartTheSameName()
+        {
+            var cfg = new IgniteConfiguration
+            {
+                SpringConfigUrl = "config\\start-test-grid1.xml",
+                JvmOptions = TestUtils.TestJavaOptions(),
+                JvmClasspath = TestUtils.CreateTestClasspath()
+            };
+
+            var grid1 = Ignition.Start(cfg);
+
+            Assert.AreEqual("grid1", grid1.Name);
+
+            try
+            {
+                Ignition.Start(cfg);
+
+                Assert.Fail("Start should fail.");
+            }
+            catch (IgniteException e)
+            {
+                Console.WriteLine("Expected exception: " + e);
+            }
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        [Test]
+        public void TestUsageAfterStop()
+        {
+            var cfg = new IgniteConfiguration
+            {
+                SpringConfigUrl = "config\\start-test-grid1.xml",
+                JvmOptions = TestUtils.TestJavaOptions(),
+                JvmClasspath = TestUtils.CreateTestClasspath()
+            };
+
+            var grid = Ignition.Start(cfg);
+
+            Assert.IsNotNull(grid.Cache<int, int>("cache1"));
+
+            grid.Dispose();
+
+            try
+            {
+                grid.Cache<int, int>("cache1");
+
+                Assert.Fail();
+            }
+            catch (InvalidOperationException e)
+            {
+                Console.WriteLine("Expected exception: " + e);
+            }
+        }
+
+        /// <summary>
+        /// 
+        /// </summary>
+        [Test]
+        public void TestStartStopLeak()
+        {
+            var cfg = new IgniteConfiguration
+            {
+                SpringConfigUrl = "config\\start-test-grid1.xml",
+                JvmOptions = new List<string> {"-Xcheck:jni", "-Xms256m", "-Xmx256m", "-XX:+HeapDumpOnOutOfMemoryError"},
+                JvmClasspath = TestUtils.CreateTestClasspath()
+            };
+
+            for (var i = 0; i < 20; i++)
+            {
+                Console.WriteLine("Iteration: " + i);
+
+                var grid = Ignition.Start(cfg);
+
+                UseIgnite(grid);
+
+                if (i % 2 == 0) // Try to stop ignite from another thread.
+                {
+                    var t = new Thread(() => {
+                        grid.Dispose();
+                    });
+
+                    t.Start();
+
+                    t.Join();
+                }
+                else
+                    grid.Dispose();
+
+                GC.Collect(); // At the time of writing java references are cleaned from finalizer, so GC is needed.
+            }
+        }
+
+        /// <summary>
+        /// Tests the client mode flag.
+        /// </summary>
+        [Test]
+        public void TestClientMode()
+        {
+            var servCfg = new IgniteConfiguration
+            {
+                SpringConfigUrl = "config\\start-test-grid1.xml",
+                JvmOptions = TestUtils.TestJavaOptions(),
+                JvmClasspath = TestUtils.CreateTestClasspath()
+            };
+
+            var clientCfg = new IgniteConfiguration
+            {
+                SpringConfigUrl = "config\\start-test-grid2.xml",
+                JvmOptions = TestUtils.TestJavaOptions(),
+                JvmClasspath = TestUtils.CreateTestClasspath()
+            };
+
+            try
+            {
+                using (Ignition.Start(servCfg))  // start server-mode ignite first
+                {
+                    Ignition.ClientMode = true;
+
+                    using (var grid = Ignition.Start(clientCfg))
+                    {
+                        UseIgnite(grid);
+                    }
+                }
+            }
+            finally 
+            {
+                Ignition.ClientMode = false;
+            }
+        }
+
+        /// <summary>
+        /// Uses the ignite.
+        /// </summary>
+        /// <param name="ignite">The ignite.</param>
+        private static void UseIgnite(IIgnite ignite)
+        {
+            // Create objects holding references to java objects.
+            var comp = ignite.Compute();
+
+            // ReSharper disable once RedundantAssignment
+            comp = comp.WithKeepPortable();
+
+            var prj = ignite.Cluster.ForOldest();
+
+            Assert.IsTrue(prj.Nodes().Count > 0);
+
+            Assert.IsNotNull(prj.Compute());
+
+            var cache = ignite.Cache<int, int>("cache1");
+
+            Assert.IsNotNull(cache);
+
+            cache.GetAndPut(1, 1);
+
+            Assert.AreEqual(1, cache.Get(1));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/IgnitionTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/IgnitionTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/IgnitionTest.cs
deleted file mode 100644
index a2698d1..0000000
--- a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/IgnitionTest.cs
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * 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 NUnit.Framework;
-
-    public class IgnitionTest
-    {
-        [Test]
-        public void Test()
-        {
-            Assert.IsNotNull(new Ignition());
-        }
-    }
-}


[15/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Services/ServiceConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Services/ServiceConfiguration.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Services/ServiceConfiguration.cs
new file mode 100644
index 0000000..e91656f
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Services/ServiceConfiguration.cs
@@ -0,0 +1,62 @@
+/*
+ * 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.Services
+{
+    using Apache.Ignite.Core.Cluster;
+
+    /// <summary>
+    /// Service configuration.
+    /// </summary>
+    public class ServiceConfiguration
+    {
+        /// <summary>
+        /// Gets or sets the service name.
+        /// </summary>
+        public string Name { get; set; }
+
+        /// <summary>
+        /// Gets or sets the service instance.
+        /// </summary>
+        public IService Service { get; set; }
+
+        /// <summary>
+        /// Gets or sets the total number of deployed service instances in the cluster, 0 for unlimited.
+        /// </summary>
+        public int TotalCount { get; set; }
+
+        /// <summary>
+        /// Gets or sets maximum number of deployed service instances on each node, 0 for unlimited.
+        /// </summary>
+        public int MaxPerNodeCount { get; set; }
+
+        /// <summary>
+        /// Gets or sets cache name used for key-to-node affinity calculation.
+        /// </summary>
+        public string CacheName { get; set; }
+
+        /// <summary>
+        /// Gets or sets affinity key used for key-to-node affinity calculation.
+        /// </summary>
+        public object AffinityKey { get; set; }
+
+        /// <summary>
+        /// Gets or sets node filter used to filter nodes on which the service will be deployed.
+        /// </summary>
+        public IClusterNodeFilter NodeFilter { get; set; } 
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Services/ServiceInvocationException.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Services/ServiceInvocationException.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Services/ServiceInvocationException.cs
new file mode 100644
index 0000000..fe83cbc
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Services/ServiceInvocationException.cs
@@ -0,0 +1,101 @@
+/*
+ * 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.Services
+{
+    using System;
+    using System.Runtime.Serialization;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Portable;
+
+    /// <summary>
+    /// Indicates an error during Grid Services invocation.
+    /// </summary>
+    [Serializable]
+    public class ServiceInvocationException : IgniteException
+    {
+        /** Serializer key. */
+        private const string KeyPortableCause = "PortableCause";
+
+        /** Cause. */
+        private readonly IPortableObject _portableCause;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ServiceInvocationException"/> class.
+        /// </summary>
+        public ServiceInvocationException()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ServiceInvocationException"/> class.
+        /// </summary>
+        /// <param name="message">The message that describes the error.</param>
+        public ServiceInvocationException(string message) : base(message)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ServiceInvocationException"/> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        public ServiceInvocationException(string message, Exception cause) : base(message, cause)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ServiceInvocationException"/> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="portableCause">The portable cause.</param>
+        public ServiceInvocationException(string message, IPortableObject portableCause)
+            :base(message)
+        {
+            _portableCause = portableCause;
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="IgniteException"/> class.
+        /// </summary>
+        /// <param name="info">Serialization information.</param>
+        /// <param name="ctx">Streaming context.</param>
+        protected ServiceInvocationException(SerializationInfo info, StreamingContext ctx)
+            : base(info, ctx)
+        {
+            _portableCause = (IPortableObject) info.GetValue(KeyPortableCause, typeof (IPortableObject));
+        }
+
+        /// <summary>
+        /// Gets the portable cause.
+        /// </summary>
+        public IPortableObject PortableCause
+        {
+            get { return _portableCause; }
+        }
+
+        /** <inheritdoc /> */
+        public override void GetObjectData(SerializationInfo info, StreamingContext context)
+        {
+            info.AddValue(KeyPortableCause, _portableCause);
+
+            base.GetObjectData(info, context);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/ITransaction.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/ITransaction.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/ITransaction.cs
new file mode 100644
index 0000000..e85d577
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/ITransaction.cs
@@ -0,0 +1,230 @@
+/*
+ * 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.Transactions
+{
+    using System;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// Grid cache transaction. 
+    /// <para />
+    /// Cache transactions support the following isolation levels:
+    /// <list type="bullet">
+    ///     <item>
+    ///         <description><see cref="TransactionIsolation.ReadCommitted"/> isolation level 
+    ///         means that always a committed value will be provided for read operations. With this isolation 
+    ///         level values are always read from cache global memory or persistent store every time a value 
+    ///         is accessed. In other words, if the same key is accessed more than once within the same transaction, 
+    ///         it may have different value every time since global cache memory may be updated concurrently by 
+    ///         other threads.</description>
+    ///     </item>
+    ///     <item>
+    ///         <description><see cref="TransactionIsolation.RepeatableRead"/> isolation level 
+    ///         means that if a value was read once within transaction, then all consecutive reads will provide 
+    ///         the same in-transaction value. With this isolation level accessed values are stored within 
+    ///         in-transaction memory, so consecutive access to the same key within the same transaction will always 
+    ///         return the value that was previously read or updated within this transaction. If concurrency is 
+    ///         <see cref="TransactionConcurrency.Pessimistic"/>, then a lock on the key will be 
+    ///         acquired prior to accessing the value.</description>
+    ///     </item>
+    ///     <item>
+    ///         <description><see cref="TransactionIsolation.Serializable"/> isolation level means 
+    ///         that all transactions occur in a completely isolated fashion, as if all transactions in the system 
+    ///         had executed serially, one after the other. Read access with this level happens the same way as with 
+    ///         <see cref="TransactionIsolation.RepeatableRead"/> level. However, in 
+    ///         <see cref="TransactionConcurrency.Optimistic"/> mode, if some transactions cannot be 
+    ///         serially isolated from each other, then one winner will be picked and the other transactions in 
+    ///         conflict will result in <c>TransactionOptimisticException</c> being thrown on Java side.</description>
+    ///     </item>
+    /// </list>
+    /// Cache transactions support the following concurrency models:
+    /// <list type="bullet">
+    ///     <item>
+    ///         <description><see cref="TransactionConcurrency.Optimistic"/> - in this mode all cache 
+    ///         operations 
+    ///         are not distributed to other nodes until <see cref="ITransaction.Commit()"/>.
+    ///         In this mode one <c>PREPARE</c> message will 
+    ///         be sent to participating cache nodes to start acquiring per-transaction locks, and once all nodes 
+    ///         reply <c>OK</c> (i.e. <c>Phase 1</c> completes successfully), a one-way <c>COMMIT</c> message is sent
+    ///         without waiting for reply. If it is necessary to know whenever remote nodes have committed as well, 
+    ///         synchronous commit or synchronous rollback should be enabled via 
+    ///         <c>CacheConfiguration.setWriteSynchronizationMode</c>.
+    ///         <para />
+    ///         Note that in this mode, optimistic failures are only possible in conjunction with
+    ///         <see cref="TransactionIsolation.Serializable"/> isolation level. In all other cases, 
+    ///         optimistic transactions will never fail optimistically and will always be identically ordered on all 
+    ///         participating Ignite nodes.</description>
+    ///     </item>
+    ///     <item>
+    ///         <description><see cref="TransactionConcurrency.Pessimistic"/> - in this mode a lock is 
+    ///         acquired on all cache operations with exception of read operations in 
+    ///         <see cref="TransactionIsolation.ReadCommitted"/> mode. All optional filters passed 
+    ///         into cache operations will be evaluated after successful lock acquisition. Whenever 
+    ///         <see cref="ITransaction.Commit()"/> is called, a single one-way <c>COMMIT</c> 
+    ///         message is sent to participating cache nodes without waiting for reply. Note that there is no reason 
+    ///         for distributed <c>PREPARE</c> step, as all locks have been already acquired. Just like with 
+    ///         optimistic mode, it is possible to configure synchronous commit or rollback and wait till 
+    ///         transaction commits on all participating remote nodes.</description>
+    ///     </item>
+    /// </list>
+    /// <para />
+    /// In addition to standard <c>CacheAtomicityMode.TRANSACTIONAL</c> behavior, Ignite also supports
+    /// a lighter <c>CacheAtomicityMode.ATOMIC</c> mode as well. In this mode distributed transactions
+    /// and distributed locking are not supported. Disabling transactions and locking allows to achieve much higher
+    /// performance and throughput ratios. It is recommended that <c>CacheAtomicityMode.TRANSACTIONAL</c> mode
+    /// is used whenever full <c>ACID</c>-compliant transactions are not needed.
+    /// <example>
+    ///     You can use cache transactions as follows:
+    ///     <code>
+    ///     ICacheTx tx = cache.TxStart();    
+    /// 
+    ///     try 
+    ///     {
+    ///         int v1 = cache&lt;string, int&gt;.Get("k1");
+    ///         
+    ///         // Check if v1 satisfies some condition before doing a put.
+    ///         if (v1 > 0)
+    ///             cache.Put&lt;string, int&gt;("k1", 2);
+    ///             
+    ///         cache.Removex("k2);
+    ///         
+    ///         // Commit the transaction.
+    ///         tx.Commit();
+    ///     }
+    ///     finally 
+    ///     {
+    ///         tx.Dispose();
+    ///     }
+    ///     
+    ///     </code>
+    /// </example>
+    /// </summary>
+    public interface ITransaction : IDisposable, IAsyncSupport<ITransaction>
+    {
+        /// <summary>
+        /// ID of the node on which this transaction started.
+        /// </summary>
+        /// <value>
+        /// Originating node ID.
+        /// </value>
+        Guid NodeId { get; }
+
+        /// <summary>
+        /// ID of the thread in which this transaction started.
+        /// </summary>
+        long ThreadId
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Start time of this transaction on this node.
+        /// </summary>
+        DateTime StartTime
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Transaction isolation level.
+        /// </summary>
+        TransactionIsolation Isolation
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Transaction concurrency mode.
+        /// </summary>
+        TransactionConcurrency Concurrency
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Current transaction state.
+        /// </summary>
+        TransactionState State
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Timeout value in milliseconds for this transaction. If transaction times
+        /// out prior to it's completion, an exception will be thrown.
+        /// </summary>
+        TimeSpan Timeout
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Gets a value indicating whether this transaction was marked as rollback-only.
+        /// </summary>
+        bool IsRollbackOnly
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Modify the transaction associated with the current thread such that the 
+        /// only possible outcome of the transaction is to roll back the transaction.
+        /// </summary>
+        /// <returns>
+        /// True if rollback-only flag was set as a result of this operation, 
+        /// false if it was already set prior to this call or could not be set
+        /// because transaction is already finishing up committing or rolling back.
+        /// </returns>
+        bool SetRollbackonly();
+
+        /// <summary>
+        /// Commits this transaction.
+        /// </summary>
+        [AsyncSupported]
+        void Commit();
+
+        /// <summary>
+        /// Rolls back this transaction.
+        /// </summary>
+        [AsyncSupported]
+        void Rollback();
+
+        /// <summary>
+        /// Adds a new metadata.
+        /// </summary>
+        /// <param name="name">Metadata name.</param>
+        /// <param name="val">Metadata value.</param>
+        void AddMeta<TV>(string name, TV val);
+
+        /// <summary>
+        /// Gets metadata by name.
+        /// </summary>
+        /// <param name="name">Metadata name.</param>
+        /// <returns>Metadata value.</returns>
+        /// <exception cref="KeyNotFoundException">If metadata key was not found.</exception>
+        TV Meta<TV>(string name);
+
+        /// <summary>
+        /// Removes metadata by name.
+        /// </summary>
+        /// <param name="name">Metadata name.</param>
+        /// <returns>Value of removed metadata or default value for <code>V</code> type.</returns>
+        TV RemoveMeta<TV>(string name);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/ITransactionMetrics.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/ITransactionMetrics.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/ITransactionMetrics.cs
new file mode 100644
index 0000000..565dd34
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/ITransactionMetrics.cs
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Transactions
+{
+    using System;
+
+    /// <summary>
+    /// Transaction metrics, shared across all caches.
+    /// </summary>
+    public interface ITransactionMetrics
+    {
+        /// <summary>
+        /// Gets the last time transaction was committed.
+        /// </summary>
+        DateTime CommitTime { get; }
+
+        /// <summary>
+        /// Gets the last time transaction was rolled back.
+        /// </summary>
+        DateTime RollbackTime { get; }
+
+        /// <summary>
+        /// Gets the total number of transaction commits.
+        /// </summary>
+        int TxCommits { get; }
+
+        /// <summary>
+        /// Gets the total number of transaction rollbacks.
+        /// </summary>
+        int TxRollbacks { get; }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/ITransactions.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/ITransactions.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/ITransactions.cs
new file mode 100644
index 0000000..83f12a5
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/ITransactions.cs
@@ -0,0 +1,73 @@
+/*
+ * 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.Transactions
+{
+    using System;
+
+    /// <summary>
+    /// Transactions facade.
+    /// <para/>
+    /// All members are thread-safe and may be used concurrently from multiple threads.
+    /// </summary>
+    public interface ITransactions
+    {
+        /// <summary>
+        /// Starts a transaction with default isolation, concurrency, timeout, and invalidation policy.
+        /// All defaults are set in CacheConfiguration at startup.
+        /// </summary>
+        /// <returns>New transaction.</returns>
+        ITransaction TxStart();
+
+        /// <summary>
+        /// Starts new transaction with the specified concurrency and isolation.
+        /// </summary>
+        /// <param name="concurrency">Concurrency.</param>
+        /// <param name="isolation">Isolation.</param>
+        /// <returns>New transaction.</returns>
+        ITransaction TxStart(TransactionConcurrency concurrency, TransactionIsolation isolation);
+
+        /// <summary>
+        /// Starts new transaction with the specified concurrency and isolation.
+        /// </summary>
+        /// <param name="concurrency">Concurrency.</param>
+        /// <param name="isolation">Isolation.</param>
+        /// <param name="timeout">Timeout.</param>
+        /// <param name="txSize">Number of entries participating in transaction (may be approximate).</param>
+        /// <returns>New transaction.</returns>
+        ITransaction TxStart(TransactionConcurrency concurrency, TransactionIsolation isolation, 
+            TimeSpan timeout, int txSize);
+
+        /// <summary>
+        /// Gets transaction started by this thread or null if this thread does not have a transaction.
+        /// </summary>
+        /// <value>
+        /// Transaction started by this thread or null if this thread does not have a transaction.
+        /// </value>
+        ITransaction Tx { get; }
+
+        /// <summary>
+        /// Gets the metrics.
+        /// </summary>
+        ITransactionMetrics GetMetrics();
+
+        /// <summary>
+        /// Resets the metrics.
+        /// </summary>
+        void ResetMetrics();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionConcurrency.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionConcurrency.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionConcurrency.cs
new file mode 100644
index 0000000..4025607
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionConcurrency.cs
@@ -0,0 +1,36 @@
+/*
+ * 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.Transactions
+{
+    /// <summary>
+    /// Transaction concurrency control. See <see cref="ITransaction"/> for more 
+    /// information on transaction concurrency controls.
+    /// </summary>
+    public enum TransactionConcurrency
+    {
+        /// <summary>
+        /// Optimistic concurrency control.
+        /// </summary>
+        Optimistic = 0,
+
+        /// <summary>
+        /// Pessimistic concurrency control.
+        /// </summary>
+        Pessimistic = 1
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionHeuristicException.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionHeuristicException.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionHeuristicException.cs
new file mode 100644
index 0000000..cb46902
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionHeuristicException.cs
@@ -0,0 +1,72 @@
+/*
+ * 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.Transactions
+{
+    using System;
+    using System.Runtime.Serialization;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary> 
+    /// Exception thrown whenever Ignite transaction enters an unknown state.
+    /// This exception is usually thrown whenever commit partially succeeds.
+    /// Cache will still resolve this situation automatically to ensure data
+    /// integrity, by invalidating all values participating in this transaction
+    /// on remote nodes.  
+    /// </summary>
+    [Serializable]
+    public class TransactionHeuristicException : IgniteException
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="TransactionHeuristicException"/> class.
+        /// </summary>
+        public TransactionHeuristicException()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="TransactionHeuristicException"/> class.
+        /// </summary>
+        /// <param name="message">The message that describes the error.</param>
+        public TransactionHeuristicException(string message) : base(message)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="TransactionHeuristicException"/> class.
+        /// </summary>
+        /// <param name="info">Serialization information.</param>
+        /// <param name="ctx">Streaming context.</param>
+        protected TransactionHeuristicException(SerializationInfo info, StreamingContext ctx)
+            : base(info, ctx)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="TransactionHeuristicException"/> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        public TransactionHeuristicException(string message, Exception cause) : base(message, cause)
+        {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionIsolation.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionIsolation.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionIsolation.cs
new file mode 100644
index 0000000..2a7723f
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionIsolation.cs
@@ -0,0 +1,41 @@
+/*
+ * 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.Transactions
+{
+    /// <summary>
+    /// Defines different cache transaction isolation levels. See <see cref="ITransaction"/>
+    /// documentation for more information about cache transaction isolation levels.
+    /// </summary>
+    public enum TransactionIsolation
+    {
+        /// <summary>
+        /// Read committed isolation level.
+        /// </summary>
+        ReadCommitted = 0,
+
+        /// <summary>
+        /// Repeatable read isolation level.
+        /// </summary>
+        RepeatableRead = 1,
+
+        /// <summary>
+        /// Serializable isolation level.
+        /// </summary>
+        Serializable = 2
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionOptimisticException.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionOptimisticException.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionOptimisticException.cs
new file mode 100644
index 0000000..2b64370
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionOptimisticException.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.Transactions
+{
+    using System;
+    using System.Runtime.Serialization;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary> 
+    /// Exception thrown whenever Ignite transactions fail optimistically.  
+    /// </summary>
+    [Serializable]
+    public class TransactionOptimisticException : IgniteException
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="TransactionOptimisticException"/> class.
+        /// </summary>
+        public TransactionOptimisticException()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="TransactionOptimisticException"/> class.
+        /// </summary>
+        /// <param name="message">The message that describes the error.</param>
+        public TransactionOptimisticException(string message) : base(message)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="TransactionOptimisticException"/> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        public TransactionOptimisticException(string message, Exception cause)
+            : base(message, cause)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="TransactionOptimisticException"/> class.
+        /// </summary>
+        /// <param name="info">Serialization information.</param>
+        /// <param name="ctx">Streaming context.</param>
+        protected TransactionOptimisticException(SerializationInfo info, StreamingContext ctx)
+            : base(info, ctx)
+        {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionRollbackException.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionRollbackException.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionRollbackException.cs
new file mode 100644
index 0000000..c1f25c8
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionRollbackException.cs
@@ -0,0 +1,68 @@
+/*
+ * 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.Transactions 
+{
+    using System;
+    using System.Runtime.Serialization;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// Exception thrown whenever Ignite transactions has been automatically rolled back.  
+    /// </summary>
+    [Serializable]
+    public class TransactionRollbackException : IgniteException
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="TransactionRollbackException"/> class.
+        /// </summary>
+        public TransactionRollbackException()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="TransactionRollbackException"/> class.
+        /// </summary>
+        /// <param name="message">The message that describes the error.</param>
+        public TransactionRollbackException(string message) : base(message)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="TransactionRollbackException"/> class.
+        /// </summary>
+        /// <param name="info">Serialization information.</param>
+        /// <param name="ctx">Streaming context.</param>
+        protected TransactionRollbackException(SerializationInfo info, StreamingContext ctx)
+            : base(info, ctx)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="TransactionRollbackException"/> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        public TransactionRollbackException(string message, Exception cause) : base(message, cause)
+        {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionState.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionState.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionState.cs
new file mode 100644
index 0000000..eecf72b
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionState.cs
@@ -0,0 +1,70 @@
+/*
+ * 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.Transactions
+{
+    /// <summary>
+    /// Cache transaction state.
+    /// </summary>
+    public enum TransactionState
+    {
+        /// <summary>
+        /// Transaction started.
+        /// </summary>
+        Active,
+
+        /// <summary>
+        /// Transaction validating.
+        /// </summary>
+        Preparing,
+
+        /// <summary>
+        /// Transaction validation succeeded.
+        /// </summary>
+        Prepared,
+
+        /// <summary>
+        /// Transaction is marked for rollback.
+        /// </summary>
+        MarkedRollback,
+
+        /// <summary>
+        /// Transaction commit started (validating finished).
+        /// </summary>
+        Committing,
+
+        /// <summary>
+        /// Transaction commit succeeded.
+        /// </summary>
+        Committed,
+        
+        /// <summary>
+        /// Transaction rollback started (validation failed).
+        /// </summary>
+        RollingBack,
+
+        /// <summary>
+        /// Transaction rollback succeeded.
+        /// </summary>
+        RolledBack,
+
+        /// <summary>
+        /// Transaction rollback failed or is otherwise unknown state.
+        /// </summary>
+        Unknown
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionTimeoutException.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionTimeoutException.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionTimeoutException.cs
new file mode 100644
index 0000000..f1e492a
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Transactions/TransactionTimeoutException.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.Transactions 
+{
+    using System;
+    using System.Runtime.Serialization;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// Exception thrown whenever Ignite transactions time out.  
+    /// </summary>
+    [Serializable]
+    public class TransactionTimeoutException : IgniteException
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="TransactionTimeoutException"/> class.
+        /// </summary>
+        public TransactionTimeoutException()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="TransactionTimeoutException"/> class.
+        /// </summary>
+        /// <param name="message">The message that describes the error.</param>
+        public TransactionTimeoutException(string message)
+            : base(message)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="TransactionTimeoutException"/> class.
+        /// </summary>
+        /// <param name="info">Serialization information.</param>
+        /// <param name="ctx">Streaming context.</param>
+        protected TransactionTimeoutException(SerializationInfo info, StreamingContext ctx)
+            : base(info, ctx)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="TransactionTimeoutException"/> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        public TransactionTimeoutException(string message, Exception cause) : base(message, cause)
+        {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.sln
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.sln b/modules/platform/src/main/dotnet/Apache.Ignite.sln
index 91bd2b6..11fc4ce 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.sln
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.sln
@@ -1,9 +1,16 @@
-Microsoft Visual Studio Solution File, Format Version 11.00
-# Visual Studio 2010
+
+Microsoft Visual Studio Solution File, Format Version 12.00
+# Visual Studio 2013
+VisualStudioVersion = 12.0.31101.0
+MinimumVisualStudioVersion = 10.0.40219.1
 Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Apache.Ignite.Core", "Apache.Ignite.Core\Apache.Ignite.Core.csproj", "{4CD2F726-7E2B-46C4-A5BA-057BB82EECB6}"
 EndProject
 Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Apache.Ignite.Core.Tests", "..\..\test\dotnet\Apache.Ignite.Core.Tests\Apache.Ignite.Core.Tests.csproj", "{6A62F66C-DA5B-4FBB-8CE7-A95F740FDC7A}"
 EndProject
+Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "common", "..\cpp\common\project\vs\common.vcxproj", "{4F7E4917-4612-4B96-9838-025711ADE391}"
+EndProject
+Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Apache.Ignite.Core.Tests.TestDll", "..\..\test\dotnet\Apache.Ignite.Core.Tests.TestDll\Apache.Ignite.Core.Tests.TestDll.csproj", "{F4A69E2D-908E-4F0F-A794-84D508D60E5F}"
+EndProject
 Global
 	GlobalSection(SolutionConfigurationPlatforms) = preSolution
 		Debug|x64 = Debug|x64
@@ -28,6 +35,22 @@ Global
 		{6A62F66C-DA5B-4FBB-8CE7-A95F740FDC7A}.Release|x64.Build.0 = Release|x64
 		{6A62F66C-DA5B-4FBB-8CE7-A95F740FDC7A}.Release|x86.ActiveCfg = Release|x86
 		{6A62F66C-DA5B-4FBB-8CE7-A95F740FDC7A}.Release|x86.Build.0 = Release|x86
+		{4F7E4917-4612-4B96-9838-025711ADE391}.Debug|x64.ActiveCfg = Debug|x64
+		{4F7E4917-4612-4B96-9838-025711ADE391}.Debug|x64.Build.0 = Debug|x64
+		{4F7E4917-4612-4B96-9838-025711ADE391}.Debug|x86.ActiveCfg = Debug|Win32
+		{4F7E4917-4612-4B96-9838-025711ADE391}.Debug|x86.Build.0 = Debug|Win32
+		{4F7E4917-4612-4B96-9838-025711ADE391}.Release|x64.ActiveCfg = Release|x64
+		{4F7E4917-4612-4B96-9838-025711ADE391}.Release|x64.Build.0 = Release|x64
+		{4F7E4917-4612-4B96-9838-025711ADE391}.Release|x86.ActiveCfg = Release|Win32
+		{4F7E4917-4612-4B96-9838-025711ADE391}.Release|x86.Build.0 = Release|Win32
+		{F4A69E2D-908E-4F0F-A794-84D508D60E5F}.Debug|x64.ActiveCfg = Debug|x64
+		{F4A69E2D-908E-4F0F-A794-84D508D60E5F}.Debug|x64.Build.0 = Debug|x64
+		{F4A69E2D-908E-4F0F-A794-84D508D60E5F}.Debug|x86.ActiveCfg = Debug|x86
+		{F4A69E2D-908E-4F0F-A794-84D508D60E5F}.Debug|x86.Build.0 = Debug|x86
+		{F4A69E2D-908E-4F0F-A794-84D508D60E5F}.Release|x64.ActiveCfg = Release|x64
+		{F4A69E2D-908E-4F0F-A794-84D508D60E5F}.Release|x64.Build.0 = Release|x64
+		{F4A69E2D-908E-4F0F-A794-84D508D60E5F}.Release|x86.ActiveCfg = Release|x86
+		{F4A69E2D-908E-4F0F-A794-84D508D60E5F}.Release|x86.Build.0 = Release|x86
 	EndGlobalSection
 	GlobalSection(SolutionProperties) = preSolution
 		HideSolutionNode = FALSE

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetConfiguration.java b/modules/platform/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetConfiguration.java
deleted file mode 100644
index 80f4b26..0000000
--- a/modules/platform/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetConfiguration.java
+++ /dev/null
@@ -1,119 +0,0 @@
-/*
- * 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.platform.dotnet;
-
-import org.apache.ignite.configuration.PlatformConfiguration;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableRawReader;
-import org.apache.ignite.portable.PortableRawWriter;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableWriter;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Mirror of .Net class Configuration.cs
- */
-public class PlatformDotNetConfiguration implements PlatformConfiguration, PortableMarshalAware {
-    /** */
-    private PlatformDotNetPortableConfiguration portableCfg;
-
-    /** */
-    private List<String> assemblies;
-
-    /**
-     * Default constructor.
-     */
-    public PlatformDotNetConfiguration() {
-        // No-op.
-    }
-
-    /**
-     * Copy constructor.
-     *
-     * @param cfg Configuration to copy.
-     */
-    public PlatformDotNetConfiguration(PlatformDotNetConfiguration cfg) {
-        if (cfg.getPortableConfiguration() != null)
-            portableCfg = new PlatformDotNetPortableConfiguration(cfg.getPortableConfiguration());
-
-        if (cfg.getAssemblies() != null)
-            assemblies = new ArrayList<>(cfg.getAssemblies());
-    }
-
-    /**
-     * @return Configuration.
-     */
-    public PlatformDotNetPortableConfiguration getPortableConfiguration() {
-        return portableCfg;
-    }
-
-    /**
-     * @param portableCfg Configuration.
-     */
-    public void setPortableConfiguration(PlatformDotNetPortableConfiguration portableCfg) {
-        this.portableCfg = portableCfg;
-    }
-
-    /**
-     * @return Assemblies.
-     */
-    public List<String> getAssemblies() {
-        return assemblies;
-    }
-
-    /**
-     *
-     * @param assemblies Assemblies.
-     */
-    public void setAssemblies(List<String> assemblies) {
-        this.assemblies = assemblies;
-    }
-
-    /**
-     * @return Configuration copy.
-     */
-    @SuppressWarnings("UnusedDeclaration")
-    private PlatformDotNetConfiguration copy() {
-        return new PlatformDotNetConfiguration(this);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writePortable(PortableWriter writer) throws PortableException {
-        PortableRawWriter rawWriter = writer.rawWriter();
-
-        rawWriter.writeObject(portableCfg);
-        rawWriter.writeCollection(assemblies);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readPortable(PortableReader reader) throws PortableException {
-        PortableRawReader rawReader = reader.rawReader();
-
-        portableCfg = rawReader.readObject();
-        assemblies = (List<String>)rawReader.<String>readCollection();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(PlatformDotNetConfiguration.class, this);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetPortableConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetPortableConfiguration.java b/modules/platform/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetPortableConfiguration.java
deleted file mode 100644
index 644a8e6..0000000
--- a/modules/platform/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetPortableConfiguration.java
+++ /dev/null
@@ -1,228 +0,0 @@
-/*
- * 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.platform.dotnet;
-
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableRawReader;
-import org.apache.ignite.portable.PortableRawWriter;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableWriter;
-
-import java.util.ArrayList;
-import java.util.Collection;
-
-/**
- * Mirror of .Net class PortableConfiguration.cs
- */
-public class PlatformDotNetPortableConfiguration implements PortableMarshalAware {
-    /** Type cfgs. */
-    private Collection<PlatformDotNetPortableTypeConfiguration> typesCfg;
-
-    /** Types. */
-    private Collection<String> types;
-
-    /** Default name mapper. */
-    private String dfltNameMapper;
-
-    /** Default id mapper. */
-    private String dfltIdMapper;
-
-    /** Default serializer. */
-    private String dfltSerializer;
-
-    /** Default metadata enabled. */
-    private boolean dfltMetadataEnabled = true;
-
-    /** Whether to cache deserialized value in IGridPortableObject */
-    private boolean dfltKeepDeserialized = true;
-
-    /**
-     * Default constructor.
-     */
-    public PlatformDotNetPortableConfiguration() {
-        // No-op.
-    }
-
-    /**
-     * Copy constructor.
-     * @param cfg configuration to copy.
-     */
-    public PlatformDotNetPortableConfiguration(PlatformDotNetPortableConfiguration cfg) {
-        if (cfg.getTypesConfiguration() != null) {
-            typesCfg = new ArrayList<>();
-
-            for (PlatformDotNetPortableTypeConfiguration typeCfg : cfg.getTypesConfiguration())
-                typesCfg.add(new PlatformDotNetPortableTypeConfiguration(typeCfg));
-        }
-
-        if (cfg.getTypes() != null)
-            types = new ArrayList<>(cfg.getTypes());
-
-        dfltNameMapper = cfg.getDefaultNameMapper();
-        dfltIdMapper = cfg.getDefaultIdMapper();
-        dfltSerializer = cfg.getDefaultSerializer();
-        dfltMetadataEnabled = cfg.isDefaultMetadataEnabled();
-        dfltKeepDeserialized = cfg.isDefaultKeepDeserialized();
-    }
-
-    /**
-     * @return Type cfgs.
-     */
-    public Collection<PlatformDotNetPortableTypeConfiguration> getTypesConfiguration() {
-        return typesCfg;
-    }
-
-    /**
-     * @param typesCfg New type cfgs.
-     */
-    public void setTypesConfiguration(Collection<PlatformDotNetPortableTypeConfiguration> typesCfg) {
-        this.typesCfg = typesCfg;
-    }
-
-    /**
-     * @return Types.
-     */
-    public Collection<String> getTypes() {
-        return types;
-    }
-
-    /**
-     * @param types New types.
-     */
-    public void setTypes(Collection<String> types) {
-        this.types = types;
-    }
-
-    /**
-     * @return Default name mapper.
-     */
-    public String getDefaultNameMapper() {
-        return dfltNameMapper;
-    }
-
-    /**
-     * @param dfltNameMapper New default name mapper.
-     */
-    public void setDefaultNameMapper(String dfltNameMapper) {
-        this.dfltNameMapper = dfltNameMapper;
-    }
-
-    /**
-     * @return Default id mapper.
-     */
-    public String getDefaultIdMapper() {
-        return dfltIdMapper;
-    }
-
-    /**
-     * @param dfltIdMapper New default id mapper.
-     */
-    public void setDefaultIdMapper(String dfltIdMapper) {
-        this.dfltIdMapper = dfltIdMapper;
-    }
-
-    /**
-     * @return Default serializer.
-     */
-    public String getDefaultSerializer() {
-        return dfltSerializer;
-    }
-
-    /**
-     * @param dfltSerializer New default serializer.
-     */
-    public void setDefaultSerializer(String dfltSerializer) {
-        this.dfltSerializer = dfltSerializer;
-    }
-
-    /**
-     * Gets default metadata enabled flag. See {@link #setDefaultMetadataEnabled(boolean)} for more information.
-     *
-     * @return Default metadata enabled flag.
-     */
-    public boolean isDefaultMetadataEnabled() {
-        return dfltMetadataEnabled;
-    }
-
-    /**
-     * Sets default metadata enabled flag. When set to {@code true} all portable types will save it's metadata to
-     * cluster.
-     * <p />
-     * Can be overridden for particular type using
-     * {@link PlatformDotNetPortableTypeConfiguration#setMetadataEnabled(Boolean)}.
-     *
-     * @param dfltMetadataEnabled Default metadata enabled flag.
-     */
-    public void setDefaultMetadataEnabled(boolean dfltMetadataEnabled) {
-        this.dfltMetadataEnabled = dfltMetadataEnabled;
-    }
-
-    /**
-     * Gets default keep deserialized flag. See {@link #setDefaultKeepDeserialized(boolean)} for more information.
-     *
-     * @return  Flag indicates whether to cache deserialized value in IGridPortableObject.
-     */
-    public boolean isDefaultKeepDeserialized() {
-        return dfltKeepDeserialized;
-    }
-
-    /**
-     * Sets default keep deserialized flag.
-     * <p />
-     * Can be overridden for particular type using
-     * {@link PlatformDotNetPortableTypeConfiguration#setKeepDeserialized(Boolean)}.
-     *
-     * @param keepDeserialized Keep deserialized flag.
-     */
-    public void setDefaultKeepDeserialized(boolean keepDeserialized) {
-        this.dfltKeepDeserialized = keepDeserialized;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writePortable(PortableWriter writer) throws PortableException {
-        PortableRawWriter rawWriter = writer.rawWriter();
-
-        rawWriter.writeCollection(typesCfg);
-        rawWriter.writeCollection(types);
-        rawWriter.writeString(dfltNameMapper);
-        rawWriter.writeString(dfltIdMapper);
-        rawWriter.writeString(dfltSerializer);
-        rawWriter.writeBoolean(dfltMetadataEnabled);
-        rawWriter.writeBoolean(dfltKeepDeserialized);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readPortable(PortableReader reader) throws PortableException {
-        PortableRawReader rawReader = reader.rawReader();
-
-        typesCfg = rawReader.readCollection();
-        types = rawReader.readCollection();
-        dfltNameMapper = rawReader.readString();
-        dfltIdMapper = rawReader.readString();
-        dfltSerializer = rawReader.readString();
-        dfltMetadataEnabled = rawReader.readBoolean();
-        dfltKeepDeserialized = rawReader.readBoolean();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(PlatformDotNetPortableConfiguration.class, this);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetPortableTypeConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetPortableTypeConfiguration.java b/modules/platform/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetPortableTypeConfiguration.java
deleted file mode 100644
index b6fdbde..0000000
--- a/modules/platform/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetPortableTypeConfiguration.java
+++ /dev/null
@@ -1,248 +0,0 @@
-/*
- * 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.platform.dotnet;
-
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableRawReader;
-import org.apache.ignite.portable.PortableRawWriter;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableWriter;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Mirror of .Net class GridDotNetPortableTypeConfiguration.cs
- */
-public class PlatformDotNetPortableTypeConfiguration implements PortableMarshalAware {
-    /** Assembly name. */
-    private String assemblyName;
-
-    /** Type name. */
-    private String typeName;
-
-    /** Name mapper. */
-    private String nameMapper;
-
-    /** Id mapper. */
-    private String idMapper;
-
-    /** Serializer. */
-    private String serializer;
-
-    /** Affinity key field name. */
-    private String affinityKeyFieldName;
-
-    /** Metadata enabled. */
-    private Boolean metadataEnabled;
-
-    /** Whether to cache deserialized value in IGridPortableObject. */
-    private Boolean keepDeserialized;
-
-    /**
-     * Default constructor.
-     */
-    public PlatformDotNetPortableTypeConfiguration() {
-        // No-op.
-    }
-
-    /**
-     * Copy constructor.
-     * @param cfg configuration to copy.
-     */
-    public PlatformDotNetPortableTypeConfiguration(PlatformDotNetPortableTypeConfiguration cfg) {
-        assemblyName = cfg.getAssemblyName();
-        typeName = cfg.getTypeName();
-        nameMapper = cfg.getNameMapper();
-        idMapper = cfg.getIdMapper();
-        serializer = cfg.getSerializer();
-        affinityKeyFieldName = cfg.getAffinityKeyFieldName();
-        metadataEnabled = cfg.getMetadataEnabled();
-        keepDeserialized = cfg.isKeepDeserialized();
-    }
-
-    /**
-     * @return Assembly name.
-     */
-    public String getAssemblyName() {
-        return assemblyName;
-    }
-
-    /**
-     * @param assemblyName New assembly name.
-     */
-    public void setAssemblyName(String assemblyName) {
-        this.assemblyName = assemblyName;
-    }
-
-    /**
-     * @return Type name.
-     */
-    public String getTypeName() {
-        return typeName;
-    }
-
-    /**
-     * @param typeName New type name.
-     */
-    public void setTypeName(String typeName) {
-        this.typeName = typeName;
-    }
-
-    /**
-     * @return Name mapper.
-     */
-    public String getNameMapper() {
-        return nameMapper;
-    }
-
-    /**
-     * @param nameMapper New name mapper.
-     */
-    public void setNameMapper(String nameMapper) {
-        this.nameMapper = nameMapper;
-    }
-
-    /**
-     * @return Id mapper.
-     */
-    public String getIdMapper() {
-        return idMapper;
-    }
-
-    /**
-     * @param idMapper New id mapper.
-     */
-    public void setIdMapper(String idMapper) {
-        this.idMapper = idMapper;
-    }
-
-    /**
-     * @return Serializer.
-     */
-    public String getSerializer() {
-        return serializer;
-    }
-
-    /**
-     * @param serializer New serializer.
-     */
-    public void setSerializer(String serializer) {
-        this.serializer = serializer;
-    }
-
-    /**
-     * Gets metadata enabled flag. See {@link #setMetadataEnabled(Boolean)} for more information.
-     *
-     * @return Metadata enabled flag.
-     */
-    public Boolean getMetadataEnabled() {
-        return metadataEnabled;
-    }
-
-    /**
-     * Sets metadata enabled flag.
-     * <p />
-     * When set to {@code null} default value taken from
-     * {@link PlatformDotNetPortableConfiguration#isDefaultMetadataEnabled()} will be used.
-     *
-     * @param metadataEnabled New metadata enabled.
-     */
-    public void setMetadataEnabled(Boolean metadataEnabled) {
-        this.metadataEnabled = metadataEnabled;
-    }
-
-    /**
-     * @return Affinity key field name.
-     */
-    public String getAffinityKeyFieldName() {
-        return affinityKeyFieldName;
-    }
-
-    /**
-     * @param affinityKeyFieldName Affinity key field name.
-     */
-    public void setAffinityKeyFieldName(String affinityKeyFieldName) {
-        this.affinityKeyFieldName = affinityKeyFieldName;
-    }
-
-    /**
-     * Gets keep deserialized flag.
-     *
-     * @return Flag indicates whether to cache deserialized value in IGridPortableObject.
-     * @deprecated Use {@link #getKeepDeserialized()} instead.
-     */
-    @Deprecated
-    @Nullable public Boolean isKeepDeserialized() {
-        return keepDeserialized;
-    }
-
-    /**
-     * Gets keep deserialized flag. See {@link #setKeepDeserialized(Boolean)} for more information.
-     *
-     * @return Flag indicates whether to cache deserialized value in IGridPortableObject.
-     */
-    @Nullable public Boolean getKeepDeserialized() {
-        return keepDeserialized;
-    }
-
-    /**
-     * Sets keep deserialized flag.
-     * <p />
-     * When set to {@code null} default value taken from
-     * {@link PlatformDotNetPortableConfiguration#isDefaultKeepDeserialized()} will be used.
-     *
-     * @param keepDeserialized Keep deserialized flag.
-     */
-    public void setKeepDeserialized(@Nullable Boolean keepDeserialized) {
-        this.keepDeserialized = keepDeserialized;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writePortable(PortableWriter writer) throws PortableException {
-        PortableRawWriter rawWriter = writer.rawWriter();
-
-        rawWriter.writeString(assemblyName);
-        rawWriter.writeString(typeName);
-        rawWriter.writeString(nameMapper);
-        rawWriter.writeString(idMapper);
-        rawWriter.writeString(serializer);
-        rawWriter.writeString(affinityKeyFieldName);
-        rawWriter.writeObject(metadataEnabled);
-        rawWriter.writeObject(keepDeserialized);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readPortable(PortableReader reader) throws PortableException {
-        PortableRawReader rawReader = reader.rawReader();
-
-        assemblyName = rawReader.readString();
-        typeName = rawReader.readString();
-        nameMapper = rawReader.readString();
-        idMapper = rawReader.readString();
-        serializer = rawReader.readString();
-        affinityKeyFieldName = rawReader.readString();
-        metadataEnabled = rawReader.readObject();
-        keepDeserialized = rawReader.readObject();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(PlatformDotNetPortableTypeConfiguration.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests.TestDll/Apache.Ignite.Core.Tests.TestDll.csproj
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests.TestDll/Apache.Ignite.Core.Tests.TestDll.csproj b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests.TestDll/Apache.Ignite.Core.Tests.TestDll.csproj
new file mode 100644
index 0000000..f213b34
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests.TestDll/Apache.Ignite.Core.Tests.TestDll.csproj
@@ -0,0 +1,52 @@
+<?xml version="1.0" encoding="utf-8"?>
+<Project ToolsVersion="12.0" DefaultTargets="Build" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
+  <Import Project="$(MSBuildExtensionsPath)\$(MSBuildToolsVersion)\Microsoft.Common.props" Condition="Exists('$(MSBuildExtensionsPath)\$(MSBuildToolsVersion)\Microsoft.Common.props')" />
+  <PropertyGroup>
+    <Configuration Condition=" '$(Configuration)' == '' ">Debug</Configuration>
+    <Platform Condition=" '$(Platform)' == '' ">AnyCPU</Platform>
+    <ProjectGuid>{F4A69E2D-908E-4F0F-A794-84D508D60E5F}</ProjectGuid>
+    <OutputType>Library</OutputType>
+    <AppDesignerFolder>Properties</AppDesignerFolder>
+    <RootNamespace>Apache.Ignite.Core.Tests.TestDll</RootNamespace>
+    <AssemblyName>Apache.Ignite.Core.Tests.TestDll</AssemblyName>
+    <TargetFrameworkVersion>v4.0</TargetFrameworkVersion>
+    <FileAlignment>512</FileAlignment>
+  </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Debug|x86'">
+    <PlatformTarget>x86</PlatformTarget>
+    <OutputPath>bin\x86\Debug\</OutputPath>
+  </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|x86'">
+    <PlatformTarget>x86</PlatformTarget>
+    <OutputPath>bin\x86\Release\</OutputPath>
+  </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Debug|x64'">
+    <PlatformTarget>x64</PlatformTarget>
+    <OutputPath>bin\x64\Debug\</OutputPath>
+  </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|x64'">
+    <PlatformTarget>x64</PlatformTarget>
+    <OutputPath>bin\x64\Release\</OutputPath>
+  </PropertyGroup>
+  <ItemGroup>
+    <Reference Include="System" />
+    <Reference Include="System.Core" />
+    <Reference Include="System.Xml.Linq" />
+    <Reference Include="System.Data.DataSetExtensions" />
+    <Reference Include="Microsoft.CSharp" />
+    <Reference Include="System.Data" />
+    <Reference Include="System.Xml" />
+  </ItemGroup>
+  <ItemGroup>
+    <Compile Include="Properties\AssemblyInfo.cs" />
+    <Compile Include="TestClass.cs" />
+  </ItemGroup>
+  <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />
+  <!-- To modify your build process, add your task inside one of the targets below and uncomment it. 
+       Other similar extension points exist, see Microsoft.Common.targets.
+  <Target Name="BeforeBuild">
+  </Target>
+  <Target Name="AfterBuild">
+  </Target>
+  -->
+</Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs
new file mode 100644
index 0000000..2401c25
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+using System.Reflection;
+using System.Runtime.InteropServices;
+
+[assembly: AssemblyTitle("Apache.Ignite.Core.Tests.TestDll")]
+[assembly: AssemblyDescription("Apache Ignite .NET Core Tests Testing Library")]
+[assembly: AssemblyConfiguration("")]
+[assembly: AssemblyCompany("Apache Software Foundation")]
+[assembly: AssemblyProduct("Apache.Ignite.Core.Tests.TestDll")]
+[assembly: AssemblyCopyright("Copyright ©  2015")]
+[assembly: AssemblyTrademark("")]
+[assembly: AssemblyCulture("")]
+
+// Setting ComVisible to false makes the types in this assembly not visible 
+// to COM components.  If you need to access a type in this assembly from 
+// COM, set the ComVisible attribute to true on that type.
+[assembly: ComVisible(false)]
+
+// The following GUID is for the ID of the typelib if this project is exposed to COM
+[assembly: Guid("086e5873-013b-4ffb-93d2-d67881f75bc2")]
+
+// Version information for an assembly consists of the following four values:
+//
+//      Major Version
+//      Minor Version 
+//      Build Number
+//      Revision
+//
+// You can specify all the values or you can default the Build and Revision Numbers 
+// by using the '*' as shown below:
+// [assembly: AssemblyVersion("1.0.*")]
+[assembly: AssemblyVersion("1.0.0.0")]
+[assembly: AssemblyFileVersion("1.0.0.0")]

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests.TestDll/TestClass.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests.TestDll/TestClass.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests.TestDll/TestClass.cs
new file mode 100644
index 0000000..1199f2c
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests.TestDll/TestClass.cs
@@ -0,0 +1,35 @@
+/*
+ * 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.TestDll
+{
+    /// <summary>
+    /// Test class.
+    /// </summary>
+    public class TestClass
+    {
+        /// <summary>
+        /// Gets or sets the Id.
+        /// </summary>
+        public int Id { get; set; }
+
+        /// <summary>
+        /// Gets or sets the Name.
+        /// </summary>
+        public string Name { get; set; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
index 418f467..62440a1 100644
--- a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
@@ -18,20 +18,27 @@
   <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Debug|x64'">
     <PlatformTarget>x64</PlatformTarget>
     <OutputPath>bin\x64\Debug\</OutputPath>
+    <AllowUnsafeBlocks>true</AllowUnsafeBlocks>
+    <DefineConstants>DEBUG</DefineConstants>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|x64'">
     <PlatformTarget>x64</PlatformTarget>
     <OutputPath>bin\x64\Release\</OutputPath>
+    <AllowUnsafeBlocks>true</AllowUnsafeBlocks>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Debug|x86'">
     <PlatformTarget>x86</PlatformTarget>
     <OutputPath>bin\x86\Debug\</OutputPath>
+    <AllowUnsafeBlocks>true</AllowUnsafeBlocks>
+    <DefineConstants>DEBUG</DefineConstants>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|x86'">
     <PlatformTarget>x86</PlatformTarget>
     <OutputPath>bin\x86\Release\</OutputPath>
+    <AllowUnsafeBlocks>true</AllowUnsafeBlocks>
   </PropertyGroup>
   <ItemGroup>
+    <Reference Include="Microsoft.CSharp" />
     <Reference Include="nunit-console-runner">
       <HintPath>..\libs\nunit-console-runner.dll</HintPath>
     </Reference>
@@ -41,18 +48,174 @@
     </Reference>
     <Reference Include="System" />
     <Reference Include="System.Core" />
+    <Reference Include="System.Runtime.Serialization" />
+    <Reference Include="System.XML" />
   </ItemGroup>
   <ItemGroup>
-    <Compile Include="IgnitionTest.cs" />
+    <Compile Include="Cache\CacheDynamicStartTest.cs" />
+    <Compile Include="Cache\CacheTestAsyncWrapper.cs" />
+    <Compile Include="Cache\CacheAbstractTest.cs" />
+    <Compile Include="Cache\CacheAffinityTest.cs" />
+    <Compile Include="Cache\CacheEntryTest.cs" />
+    <Compile Include="Cache\CacheForkedTest.cs" />
+    <Compile Include="Cache\CacheLocalAtomicTest.cs" />
+    <Compile Include="Cache\CacheLocalTest.cs" />
+    <Compile Include="Cache\CachePartitionedAtomicNearEnabledTest.cs" />
+    <Compile Include="Cache\CachePartitionedAtomicTest.cs" />
+    <Compile Include="Cache\CachePartitionedNearEnabledTest.cs" />
+    <Compile Include="Cache\CachePartitionedTest.cs" />
+    <Compile Include="Cache\CacheReplicatedAtomicTest.cs" />
+    <Compile Include="Cache\CacheReplicatedTest.cs" />
+    <Compile Include="Cache\Query\Continuous\ContinuousQueryAbstractTest.cs" />
+    <Compile Include="Cache\Query\Continuous\ContinuousQueryAtomicBackupTest.cs" />
+    <Compile Include="Cache\Query\Continuous\ContinuousQueryAtomicNoBackupTest.cs" />
+    <Compile Include="Cache\Query\Continuous\ContinuousQueryNoBackupAbstractTest.cs" />
+    <Compile Include="Cache\Query\Continuous\ContinuousQueryTransactionalBackupTest.cs" />
+    <Compile Include="Cache\Query\Continuous\ContinuousQueryTransactionalNoBackupTest.cs" />
+    <Compile Include="Cache\Query\CacheQueriesTest.cs" />
+    <Compile Include="Cache\Store\CacheParallelLoadStoreTest.cs" />
+    <Compile Include="Cache\Store\CacheStoreSessionTest.cs" />
+    <Compile Include="Cache\Store\CacheStoreTest.cs" />
+    <Compile Include="Cache\Store\CacheTestParallelLoadStore.cs" />
+    <Compile Include="Cache\Store\CacheTestStore.cs" />
+    <Compile Include="Compute\Forked\ForkedPortableClosureTaskTest.cs" />
+    <Compile Include="Compute\Forked\ForkedResourceTaskTest.cs" />
+    <Compile Include="Compute\Forked\ForkedSerializableClosureTaskTest.cs" />
+    <Compile Include="Compute\Forked\ForkedTaskAdapterTest.cs" />
+    <Compile Include="Compute\AbstractTaskTest.cs" />
+    <Compile Include="Compute\ClosureTaskTest.cs" />
+    <Compile Include="Compute\ComputeApiTest.cs" />
+    <Compile Include="Compute\ComputeMultithreadedTest.cs" />
+    <Compile Include="Compute\IgniteExceptionTaskSelfTest.cs" />
+    <Compile Include="Compute\FailoverTaskSelfTest.cs" />
+    <Compile Include="Compute\PortableClosureTaskTest.cs" />
+    <Compile Include="Compute\PortableTaskTest.cs" />
+    <Compile Include="Compute\ResourceTaskTest.cs" />
+    <Compile Include="Compute\SerializableClosureTaskTest.cs" />
+    <Compile Include="Compute\TaskAdapterTest.cs" />
+    <Compile Include="Compute\TaskResultTest.cs" />
+    <Compile Include="Dataload\DataStreamerTest.cs" />
+    <Compile Include="EventsTest.cs" />
+    <Compile Include="ExceptionsTest.cs" />
+    <Compile Include="ExecutableTest.cs" />
+    <Compile Include="FutureTest.cs" />
+    <Compile Include="LifecycleTest.cs" />
+    <Compile Include="LoadDllTest.cs" />
+    <Compile Include="IgniteManagerTest.cs" />
+    <Compile Include="MarshallerTest.cs" />
+    <Compile Include="MessagingTest.cs" />
+    <Compile Include="PortableConfigurationTest.cs" />
+    <Compile Include="SerializationTest.cs" />
+    <Compile Include="IgniteStartStopTest.cs" />
+    <Compile Include="TestUtils.cs" />
     <Compile Include="Memory\InteropMemoryTest.cs" />
+    <Compile Include="Portable\PortableApiSelfTest.cs" />
+    <Compile Include="Portable\PortableSelfTest.cs" />
+    <Compile Include="Process\IgniteProcess.cs" />
+    <Compile Include="Process\IgniteProcessConsoleOutputReader.cs" />
+    <Compile Include="Process\IIgniteProcessOutputReader.cs" />
     <Compile Include="Properties\AssemblyInfo.cs" />
+    <Compile Include="Query\ImplicitPortablePerson.cs" />
+    <Compile Include="Query\NoDefPortablePerson.cs" />
+    <Compile Include="Query\PortablePerson.cs" />
+    <Compile Include="Services\ServicesTest.cs" />
+    <Compile Include="Services\ServicesTestAsync.cs" />
+    <Compile Include="Services\ServiceProxyTest.cs" />
+    <Compile Include="Services\ServicesAsyncWrapper.cs" />
     <Compile Include="TestRunner.cs" />
+    <Compile Include="TypeResolverTest.cs" />
   </ItemGroup>
   <ItemGroup>
     <ProjectReference Include="..\..\..\main\dotnet\Apache.Ignite.Core\Apache.Ignite.Core.csproj">
       <Project>{4CD2F726-7E2B-46C4-A5BA-057BB82EECB6}</Project>
       <Name>Apache.Ignite.Core</Name>
     </ProjectReference>
+    <ProjectReference Include="..\Apache.Ignite.Core.Tests.TestDll\Apache.Ignite.Core.Tests.TestDll.csproj">
+      <Project>{F4A69E2D-908E-4F0F-A794-84D508D60E5F}</Project>
+      <Name>Apache.Ignite.Core.Tests.TestDll</Name>
+    </ProjectReference>
+  </ItemGroup>
+  <ItemGroup>
+    <Content Include="Config\cache-portables.xml">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
+    <Content Include="Config\cache-query-continuous.xml">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
+    <Content Include="Config\cache-query.xml">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
+    <Content Include="Config\Cache\Store\cache-store-session.xml">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
+    <Content Include="Config\Compute\compute-grid1.xml">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
+    <Content Include="Config\Compute\compute-grid2.xml">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
+    <Content Include="Config\Compute\compute-grid3.xml">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
+    <Content Include="Config\Compute\compute-standalone.xml">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
+    <Content Include="Config\Dynamic\dynamic-client.xml">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
+    <Content Include="Config\Dynamic\dynamic-data-no-cfg.xml">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
+    <Content Include="Config\Dynamic\dynamic-data.xml">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
+    <Content Include="Config\Lifecycle\lifecycle-beans.xml">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
+    <Content Include="Config\Lifecycle\lifecycle-no-beans.xml">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
+    <Content Include="Config\marshaller-default.xml">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
+    <Content Include="Config\marshaller-invalid.xml">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
+    <Content Include="Config\marshaller-portable.xml">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
+    <Content Include="Config\native-client-test-cache-affinity.xml">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
+    <Content Include="Config\native-client-test-cache-parallel-store.xml">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
+    <Content Include="Config\native-client-test-cache-portables.xml">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
+    <Content Include="Config\native-client-test-cache-store.xml">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
+    <Content Include="Config\native-client-test-cache.xml">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
+    <Content Include="Config\portable.xml">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
+    <Content Include="Config\start-test-grid1.xml">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
+    <Content Include="Config\start-test-grid2.xml">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
+    <Content Include="Config\start-test-grid3.xml">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
+  </ItemGroup>
+  <ItemGroup>
+    <Content Include="Config\Ignite.exe.config.test">
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
   </ItemGroup>
   <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />
   <!-- To modify your build process, add your task inside one of the targets below and uncomment it. 


[17/45] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
new file mode 100644
index 0000000..9ec2668
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
@@ -0,0 +1,1252 @@
+/*
+ * 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.Unmanaged
+{
+    using System;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Runtime.InteropServices;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Common;
+
+    /// <summary>
+    /// Unmanaged utility classes.
+    /// </summary>
+    internal static unsafe class UnmanagedUtils
+    {
+        /** Interop factory ID for .Net. */
+        private const int InteropFactoryId = 1;
+
+        #region PROCEDURE NAMES
+
+        private const string ProcReallocate = "IgniteReallocate";
+
+        private const string ProcIgnitionStart = "IgniteIgnitionStart";
+        private const string ProcIgnitionStop = "IgniteIgnitionStop";
+        private const string ProcIgnitionStopAll = "IgniteIgnitionStopAll";
+        
+        private const string ProcProcessorReleaseStart = "IgniteProcessorReleaseStart";
+        private const string ProcProcessorProjection = "IgniteProcessorProjection";
+        private const string ProcProcessorCache = "IgniteProcessorCache";
+        private const string ProcProcessorGetOrCreateCache = "IgniteProcessorGetOrCreateCache";
+        private const string ProcProcessorCreateCache = "IgniteProcessorCreateCache";
+        private const string ProcProcessorAffinity = "IgniteProcessorAffinity";
+        private const string ProcProcessorDataStreamer = "IgniteProcessorDataStreamer";
+        private const string ProcProcessorTransactions = "IgniteProcessorTransactions";
+        private const string ProcProcessorCompute = "IgniteProcessorCompute";
+        private const string ProcProcessorMessage = "IgniteProcessorMessage";
+        private const string ProcProcessorEvents = "IgniteProcessorEvents";
+        private const string ProcProcessorServices = "IgniteProcessorServices";
+        private const string ProcProcessorExtensions = "IgniteProcessorExtensions";
+        
+        private const string ProcTargetInStreamOutLong = "IgniteTargetInStreamOutLong";
+        private const string ProcTargetInStreamOutStream = "IgniteTargetInStreamOutStream";
+        private const string ProcTargetInStreamOutObject = "IgniteTargetInStreamOutObject";
+        private const string ProcTargetInObjectStreamOutStream = "IgniteTargetInObjectStreamOutStream";
+        private const string ProcTargetOutLong = "IgniteTargetOutLong";
+        private const string ProcTargetOutStream = "IgniteTargetOutStream";
+        private const string ProcTargetOutObject = "IgniteTargetOutObject";
+        private const string ProcTargetListenFut = "IgniteTargetListenFuture";
+        private const string ProcTargetListenFutForOp = "IgniteTargetListenFutureForOperation";
+
+        private const string ProcAffinityParts = "IgniteAffinityPartitions";
+
+        private const string ProcCacheWithSkipStore = "IgniteCacheWithSkipStore";
+        private const string ProcCacheWithNoRetries = "IgniteCacheWithNoRetries";
+        private const string ProcCacheWithExpiryPolicy = "IgniteCacheWithExpiryPolicy";
+        private const string ProcCacheWithAsync = "IgniteCacheWithAsync";
+        private const string ProcCacheWithKeepPortable = "IgniteCacheWithKeepPortable";
+        private const string ProcCacheClear = "IgniteCacheClear";
+        private const string ProcCacheRemoveAll = "IgniteCacheRemoveAll";
+        private const string ProcCacheOutOpQueryCursor = "IgniteCacheOutOpQueryCursor";
+        private const string ProcCacheOutOpContinuousQuery = "IgniteCacheOutOpContinuousQuery";
+        private const string ProcCacheIterator = "IgniteCacheIterator";
+        private const string ProcCacheLocalIterator = "IgniteCacheLocalIterator";
+        private const string ProcCacheEnterLock = "IgniteCacheEnterLock";
+        private const string ProcCacheExitLock = "IgniteCacheExitLock";
+        private const string ProcCacheTryEnterLock = "IgniteCacheTryEnterLock";
+        private const string ProcCacheCloseLock = "IgniteCacheCloseLock";
+        private const string ProcCacheRebalance = "IgniteCacheRebalance";
+        private const string ProcCacheSize = "IgniteCacheSize";
+
+        private const string ProcCacheStoreCallbackInvoke = "IgniteCacheStoreCallbackInvoke";
+
+        private const string ProcComputeWithNoFailover = "IgniteComputeWithNoFailover";
+        private const string ProcComputeWithTimeout = "IgniteComputeWithTimeout";
+        private const string ProcComputeExecuteNative = "IgniteComputeExecuteNative";
+
+        private const string ProcContinuousQryClose = "IgniteContinuousQueryClose";
+        private const string ProcContinuousQryGetInitialQueryCursor = "IgniteContinuousQueryGetInitialQueryCursor";
+
+        private const string ProcDataStreamerListenTop = "IgniteDataStreamerListenTopology";
+        private const string ProcDataStreamerAllowOverwriteGet = "IgniteDataStreamerAllowOverwriteGet";
+        private const string ProcDataStreamerAllowOverwriteSet = "IgniteDataStreamerAllowOverwriteSet";
+        private const string ProcDataStreamerSkipStoreGet = "IgniteDataStreamerSkipStoreGet";
+        private const string ProcDataStreamerSkipStoreSet = "IgniteDataStreamerSkipStoreSet";
+        private const string ProcDataStreamerPerNodeBufferSizeGet = "IgniteDataStreamerPerNodeBufferSizeGet";
+        private const string ProcDataStreamerPerNodeBufferSizeSet = "IgniteDataStreamerPerNodeBufferSizeSet";
+        private const string ProcDataStreamerPerNodeParallelOpsGet = "IgniteDataStreamerPerNodeParallelOperationsGet";
+        private const string ProcDataStreamerPerNodeParallelOpsSet = "IgniteDataStreamerPerNodeParallelOperationsSet";
+
+        private const string ProcMessagingWithAsync = "IgniteMessagingWithAsync";
+
+        private const string ProcQryCursorIterator = "IgniteQueryCursorIterator";
+        private const string ProcQryCursorClose = "IgniteQueryCursorClose";
+
+        private const string ProcProjectionForOthers = "IgniteProjectionForOthers";
+        private const string ProcProjectionForRemotes = "IgniteProjectionForRemotes";
+        private const string ProcProjectionForDaemons = "IgniteProjectionForDaemons";
+        private const string ProcProjectionForRandom = "IgniteProjectionForRandom";
+        private const string ProcProjectionForOldest = "IgniteProjectionForOldest";
+        private const string ProcProjectionForYoungest = "IgniteProjectionForYoungest";
+        private const string ProcProjectionResetMetrics = "IgniteProjectionResetMetrics";
+        private const string ProcProjectionOutOpRet = "IgniteProjectionOutOpRet";
+
+        private const string ProcRelease = "IgniteRelease";
+
+        private const string ProcTxStart = "IgniteTransactionsStart";
+        private const string ProcTxCommit = "IgniteTransactionsCommit";
+        private const string ProcTxCommitAsync = "IgniteTransactionsCommitAsync";
+        private const string ProcTxRollback = "IgniteTransactionsRollback";
+        private const string ProcTxRollbackAsync = "IgniteTransactionsRollbackAsync";
+        private const string ProcTxClose = "IgniteTransactionsClose";
+        private const string ProcTxState = "IgniteTransactionsState";
+        private const string ProcTxSetRollbackOnly = "IgniteTransactionsSetRollbackOnly";
+        private const string ProcTxResetMetrics = "IgniteTransactionsResetMetrics";
+
+        private const string ProcThrowToJava = "IgniteThrowToJava";
+
+        private const string ProcDestroyJvm = "IgniteDestroyJvm";
+
+        private const string ProcHandlersSize = "IgniteHandlersSize";
+
+        private const string ProcCreateContext = "IgniteCreateContext";
+        
+        private const string ProcEventsWithAsync = "IgniteEventsWithAsync";
+        private const string ProcEventsStopLocalListen = "IgniteEventsStopLocalListen";
+        private const string ProcEventsLocalListen = "IgniteEventsLocalListen";
+        private const string ProcEventsIsEnabled = "IgniteEventsIsEnabled";
+
+        private const string ProcDeleteContext = "IgniteDeleteContext";
+        
+        private const string ProcServicesWithAsync = "IgniteServicesWithAsync";
+        private const string ProcServicesWithServerKeepPortable = "IgniteServicesWithServerKeepPortable";
+        private const string ProcServicesCancel = "IgniteServicesCancel";
+        private const string ProcServicesCancelAll = "IgniteServicesCancelAll";
+        private const string ProcServicesGetServiceProxy = "IgniteServicesGetServiceProxy";
+        
+        #endregion
+
+        #region DELEGATE DEFINITIONS
+
+        private delegate int ReallocateDelegate(long memPtr, int cap);
+
+        private delegate void* IgnitionStartDelegate(void* ctx, sbyte* cfgPath, sbyte* gridName, int factoryId, long dataPtr);
+        private delegate bool IgnitionStopDelegate(void* ctx, sbyte* gridName, bool cancel);
+        private delegate void IgnitionStopAllDelegate(void* ctx, bool cancel);
+
+        private delegate void ProcessorReleaseStartDelegate(void* ctx, void* obj);
+        private delegate void* ProcessorProjectionDelegate(void* ctx, void* obj);
+        private delegate void* ProcessorCacheDelegate(void* ctx, void* obj, sbyte* name);
+        private delegate void* ProcessorCreateCacheDelegate(void* ctx, void* obj, sbyte* name);
+        private delegate void* ProcessorGetOrCreateCacheDelegate(void* ctx, void* obj, sbyte* name);
+        private delegate void* ProcessorAffinityDelegate(void* ctx, void* obj, sbyte* name);
+        private delegate void* ProcessorDataStreamerDelegate(void* ctx, void* obj, sbyte* name, bool keepPortable);
+        private delegate void* ProcessorTransactionsDelegate(void* ctx, void* obj);
+        private delegate void* ProcessorComputeDelegate(void* ctx, void* obj, void* prj);
+        private delegate void* ProcessorMessageDelegate(void* ctx, void* obj, void* prj);
+        private delegate void* ProcessorEventsDelegate(void* ctx, void* obj, void* prj);
+        private delegate void* ProcessorServicesDelegate(void* ctx, void* obj, void* prj);
+        private delegate void* ProcessorExtensionsDelegate(void* ctx, void* obj);
+        
+        private delegate long TargetInStreamOutLongDelegate(void* ctx, void* target, int opType, long memPtr);
+        private delegate void TargetInStreamOutStreamDelegate(void* ctx, void* target, int opType, long inMemPtr, long outMemPtr);
+        private delegate void* TargetInStreamOutObjectDelegate(void* ctx, void* target, int opType, long memPtr);
+        private delegate void TargetInObjectStreamOutStreamDelegate(void* ctx, void* target, int opType, void* arg, long inMemPtr, long outMemPtr);
+        private delegate long TargetOutLongDelegate(void* ctx, void* target, int opType);
+        private delegate void TargetOutStreamDelegate(void* ctx, void* target, int opType, long memPtr);
+        private delegate void* TargetOutObjectDelegate(void* ctx, void* target, int opType);
+        private delegate void TargetListenFutureDelegate(void* ctx, void* target, long futId, int typ);
+        private delegate void TargetListenFutureForOpDelegate(void* ctx, void* target, long futId, int typ, int opId);
+
+        private delegate int AffinityPartitionsDelegate(void* ctx, void* target);
+
+        private delegate void* CacheWithSkipStoreDelegate(void* ctx, void* obj);
+        private delegate void* CacheNoRetriesDelegate(void* ctx, void* obj);
+        private delegate void* CacheWithExpiryPolicyDelegate(void* ctx, void* obj, long create, long update, long access);
+        private delegate void* CacheWithAsyncDelegate(void* ctx, void* obj);
+        private delegate void* CacheWithKeepPortableDelegate(void* ctx, void* obj);
+        private delegate void CacheClearDelegate(void* ctx, void* obj);
+        private delegate void CacheRemoveAllDelegate(void* ctx, void* obj);
+        private delegate void* CacheOutOpQueryCursorDelegate(void* ctx, void* obj, int type, long memPtr);
+        private delegate void* CacheOutOpContinuousQueryDelegate(void* ctx, void* obj, int type, long memPtr);
+        private delegate void* CacheIteratorDelegate(void* ctx, void* obj);
+        private delegate void* CacheLocalIteratorDelegate(void* ctx, void* obj, int peekModes);
+        private delegate void CacheEnterLockDelegate(void* ctx, void* obj, long id);
+        private delegate void CacheExitLockDelegate(void* ctx, void* obj, long id);
+        private delegate bool CacheTryEnterLockDelegate(void* ctx, void* obj, long id, long timeout);
+        private delegate void CacheCloseLockDelegate(void* ctx, void* obj, long id);
+        private delegate void CacheRebalanceDelegate(void* ctx, void* obj, long futId);
+        private delegate int CacheSizeDelegate(void* ctx, void* obj, int peekModes, bool loc);
+
+        private delegate void CacheStoreCallbackInvokeDelegate(void* ctx, void* obj, long memPtr);
+
+        private delegate void ComputeWithNoFailoverDelegate(void* ctx, void* target);
+        private delegate void ComputeWithTimeoutDelegate(void* ctx, void* target, long timeout);
+        private delegate void ComputeExecuteNativeDelegate(void* ctx, void* target, long taskPtr, long topVer);
+
+        private delegate void ContinuousQueryCloseDelegate(void* ctx, void* target);
+        private delegate void* ContinuousQueryGetInitialQueryCursorDelegate(void* ctx, void* target);
+
+        private delegate void DataStreamerListenTopologyDelegate(void* ctx, void* obj, long ptr);
+        private delegate bool DataStreamerAllowOverwriteGetDelegate(void* ctx, void* obj);
+        private delegate void DataStreamerAllowOverwriteSetDelegate(void* ctx, void* obj, bool val);
+        private delegate bool DataStreamerSkipStoreGetDelegate(void* ctx, void* obj);
+        private delegate void DataStreamerSkipStoreSetDelegate(void* ctx, void* obj, bool val);
+        private delegate int DataStreamerPerNodeBufferSizeGetDelegate(void* ctx, void* obj);
+        private delegate void DataStreamerPerNodeBufferSizeSetDelegate(void* ctx, void* obj, int val);
+        private delegate int DataStreamerPerNodeParallelOperationsGetDelegate(void* ctx, void* obj);
+        private delegate void DataStreamerPerNodeParallelOperationsSetDelegate(void* ctx, void* obj, int val);
+
+        private delegate void* MessagingWithAsyncDelegate(void* ctx, void* target);
+
+        private delegate void* ProjectionForOthersDelegate(void* ctx, void* obj, void* prj);
+		private delegate void* ProjectionForRemotesDelegate(void* ctx, void* obj);
+		private delegate void* ProjectionForDaemonsDelegate(void* ctx, void* obj);
+		private delegate void* ProjectionForRandomDelegate(void* ctx, void* obj);
+		private delegate void* ProjectionForOldestDelegate(void* ctx, void* obj);
+		private delegate void* ProjectionForYoungestDelegate(void* ctx, void* obj);
+		private delegate void ProjectionResetMetricsDelegate(void* ctx, void* obj);
+		private delegate void* ProjectionOutOpRetDelegate(void* ctx, void* obj, int type, long memPtr);
+
+        private delegate void QueryCursorIteratorDelegate(void* ctx, void* target);
+        private delegate void QueryCursorCloseDelegate(void* ctx, void* target);
+
+        private delegate void ReleaseDelegate(void* target);
+
+        private delegate long TransactionsStartDelegate(void* ctx, void* target, int concurrency, int isolation, long timeout, int txSize);
+        private delegate int TransactionsCommitDelegate(void* ctx, void* target, long id);
+        private delegate void TransactionsCommitAsyncDelegate(void* ctx, void* target, long id, long futId);
+        private delegate int TransactionsRollbackDelegate(void* ctx, void* target, long id);
+        private delegate void TransactionsRollbackAsyncDelegate(void* ctx, void* target, long id, long futId);
+        private delegate int TransactionsCloseDelegate(void* ctx, void* target, long id);
+        private delegate int TransactionsStateDelegate(void* ctx, void* target, long id);
+        private delegate bool TransactionsSetRollbackOnlyDelegate(void* ctx, void* target, long id);
+        private delegate void TransactionsResetMetricsDelegate(void* ctx, void* target);
+
+        private delegate void ThrowToJavaDelegate(void* ctx, char* msg);
+
+        private delegate void DestroyJvmDelegate(void* ctx);
+
+        private delegate int HandlersSizeDelegate();
+
+        private delegate void* CreateContextDelegate(void* opts, int optsLen, void* cbs);
+        
+        private delegate void* EventsWithAsyncDelegate(void* ctx, void* obj);
+        private delegate bool EventsStopLocalListenDelegate(void* ctx, void* obj, long hnd);
+        private delegate void EventsLocalListenDelegate(void* ctx, void* obj, long hnd, int type);
+        private delegate bool EventsIsEnabledDelegate(void* ctx, void* obj, int type);
+
+        private delegate void DeleteContextDelegate(void* ptr);
+
+        private delegate void* ServicesWithAsyncDelegate(void* ctx, void* target);
+        private delegate void* ServicesWithServerKeepPortableDelegate(void* ctx, void* target);
+        private delegate long ServicesCancelDelegate(void* ctx, void* target, char* name);
+        private delegate long ServicesCancelAllDelegate(void* ctx, void* target);
+        private delegate void* ServicesGetServiceProxyDelegate(void* ctx, void* target, char* name, bool sticky);
+
+        #endregion
+
+        #region DELEGATE MEMBERS
+
+        // ReSharper disable InconsistentNaming
+        private static readonly ReallocateDelegate REALLOCATE;
+
+        private static readonly IgnitionStartDelegate IGNITION_START;
+        private static readonly IgnitionStopDelegate IGNITION_STOP;
+        private static readonly IgnitionStopAllDelegate IGNITION_STOP_ALL;
+
+        private static readonly ProcessorReleaseStartDelegate PROCESSOR_RELEASE_START;
+        private static readonly ProcessorProjectionDelegate PROCESSOR_PROJECTION;
+        private static readonly ProcessorCacheDelegate PROCESSOR_CACHE;
+        private static readonly ProcessorCreateCacheDelegate PROCESSOR_CREATE_CACHE;
+        private static readonly ProcessorGetOrCreateCacheDelegate PROCESSOR_GET_OR_CREATE_CACHE;
+        private static readonly ProcessorAffinityDelegate PROCESSOR_AFFINITY;
+        private static readonly ProcessorDataStreamerDelegate PROCESSOR_DATA_STREAMER;
+        private static readonly ProcessorTransactionsDelegate PROCESSOR_TRANSACTIONS;
+        private static readonly ProcessorComputeDelegate PROCESSOR_COMPUTE;
+        private static readonly ProcessorMessageDelegate PROCESSOR_MESSAGE;
+        private static readonly ProcessorEventsDelegate PROCESSOR_EVENTS;
+        private static readonly ProcessorServicesDelegate PROCESSOR_SERVICES;
+        private static readonly ProcessorExtensionsDelegate PROCESSOR_EXTENSIONS;
+        
+        private static readonly TargetInStreamOutLongDelegate TARGET_IN_STREAM_OUT_LONG;
+        private static readonly TargetInStreamOutStreamDelegate TARGET_IN_STREAM_OUT_STREAM;
+        private static readonly TargetInStreamOutObjectDelegate TARGET_IN_STREAM_OUT_OBJECT;
+        private static readonly TargetInObjectStreamOutStreamDelegate TARGET_IN_OBJECT_STREAM_OUT_STREAM;
+        private static readonly TargetOutLongDelegate TARGET_OUT_LONG;
+        private static readonly TargetOutStreamDelegate TARGET_OUT_STREAM;
+        private static readonly TargetOutObjectDelegate TARGET_OUT_OBJECT;
+        private static readonly TargetListenFutureDelegate TargetListenFut;
+        private static readonly TargetListenFutureForOpDelegate TargetListenFutForOp;
+
+        private static readonly AffinityPartitionsDelegate AffinityParts;
+
+        private static readonly CacheWithSkipStoreDelegate CACHE_WITH_SKIP_STORE;
+        private static readonly CacheNoRetriesDelegate CACHE_WITH_NO_RETRIES;
+        private static readonly CacheWithExpiryPolicyDelegate CACHE_WITH_EXPIRY_POLICY;
+        private static readonly CacheWithAsyncDelegate CACHE_WITH_ASYNC;
+        private static readonly CacheWithKeepPortableDelegate CACHE_WITH_KEEP_PORTABLE;
+        private static readonly CacheClearDelegate CACHE_CLEAR;
+        private static readonly CacheRemoveAllDelegate CACHE_REMOVE_ALL;
+        private static readonly CacheOutOpQueryCursorDelegate CACHE_OUT_OP_QUERY_CURSOR;
+        private static readonly CacheOutOpContinuousQueryDelegate CACHE_OUT_OP_CONTINUOUS_QUERY;
+        private static readonly CacheIteratorDelegate CACHE_ITERATOR;
+        private static readonly CacheLocalIteratorDelegate CACHE_LOCAL_ITERATOR;
+        private static readonly CacheEnterLockDelegate CACHE_ENTER_LOCK;
+        private static readonly CacheExitLockDelegate CACHE_EXIT_LOCK;
+        private static readonly CacheTryEnterLockDelegate CACHE_TRY_ENTER_LOCK;
+        private static readonly CacheCloseLockDelegate CACHE_CLOSE_LOCK;
+        private static readonly CacheRebalanceDelegate CACHE_REBALANCE;
+        private static readonly CacheSizeDelegate CACHE_SIZE;
+
+        private static readonly CacheStoreCallbackInvokeDelegate CACHE_STORE_CALLBACK_INVOKE;
+
+        private static readonly ComputeWithNoFailoverDelegate COMPUTE_WITH_NO_FAILOVER;
+        private static readonly ComputeWithTimeoutDelegate COMPUTE_WITH_TIMEOUT;
+        private static readonly ComputeExecuteNativeDelegate COMPUTE_EXECUTE_NATIVE;
+
+        private static readonly ContinuousQueryCloseDelegate ContinuousQryClose;
+        private static readonly ContinuousQueryGetInitialQueryCursorDelegate ContinuousQryGetInitialQueryCursor;
+
+        private static readonly DataStreamerListenTopologyDelegate DataStreamerListenTop;
+        private static readonly DataStreamerAllowOverwriteGetDelegate DATA_STREAMER_ALLOW_OVERWRITE_GET;
+        private static readonly DataStreamerAllowOverwriteSetDelegate DATA_STREAMER_ALLOW_OVERWRITE_SET;
+        private static readonly DataStreamerSkipStoreGetDelegate DATA_STREAMER_SKIP_STORE_GET;
+        private static readonly DataStreamerSkipStoreSetDelegate DATA_STREAMER_SKIP_STORE_SET;
+        private static readonly DataStreamerPerNodeBufferSizeGetDelegate DATA_STREAMER_PER_NODE_BUFFER_SIZE_GET;
+        private static readonly DataStreamerPerNodeBufferSizeSetDelegate DATA_STREAMER_PER_NODE_BUFFER_SIZE_SET;
+        private static readonly DataStreamerPerNodeParallelOperationsGetDelegate DataStreamerPerNodeParallelOpsGet;
+        private static readonly DataStreamerPerNodeParallelOperationsSetDelegate DataStreamerPerNodeParallelOpsSet;
+
+        private static readonly MessagingWithAsyncDelegate MessagingWithAsync;
+
+        private static readonly ProjectionForOthersDelegate PROJECTION_FOR_OTHERS;
+        private static readonly ProjectionForRemotesDelegate PROJECTION_FOR_REMOTES;
+        private static readonly ProjectionForDaemonsDelegate PROJECTION_FOR_DAEMONS;
+        private static readonly ProjectionForRandomDelegate PROJECTION_FOR_RANDOM;
+        private static readonly ProjectionForOldestDelegate PROJECTION_FOR_OLDEST;
+        private static readonly ProjectionForYoungestDelegate PROJECTION_FOR_YOUNGEST;
+        private static readonly ProjectionResetMetricsDelegate PROJECTION_RESET_METRICS;
+        private static readonly ProjectionOutOpRetDelegate PROJECTION_OUT_OP_RET;
+
+        private static readonly QueryCursorIteratorDelegate QryCursorIterator;
+        private static readonly QueryCursorCloseDelegate QryCursorClose;
+
+        private static readonly ReleaseDelegate RELEASE;
+
+        private static readonly TransactionsStartDelegate TxStart;
+        private static readonly TransactionsCommitDelegate TxCommit;
+        private static readonly TransactionsCommitAsyncDelegate TxCommitAsync;
+        private static readonly TransactionsRollbackDelegate TxRollback;
+        private static readonly TransactionsRollbackAsyncDelegate TxRollbackAsync;
+        private static readonly TransactionsCloseDelegate TxClose;
+        private static readonly TransactionsStateDelegate TxState;
+        private static readonly TransactionsSetRollbackOnlyDelegate TxSetRollbackOnly;
+        private static readonly TransactionsResetMetricsDelegate TxResetMetrics;
+
+        private static readonly ThrowToJavaDelegate THROW_TO_JAVA;
+
+        private static readonly DestroyJvmDelegate DESTROY_JVM;
+
+        private static readonly HandlersSizeDelegate HANDLERS_SIZE;
+
+        private static readonly CreateContextDelegate CREATE_CONTEXT;
+        
+        private static readonly EventsWithAsyncDelegate EVENTS_WITH_ASYNC;
+        private static readonly EventsStopLocalListenDelegate EVENTS_STOP_LOCAL_LISTEN;
+        private static readonly EventsLocalListenDelegate EVENTS_LOCAL_LISTEN;
+        private static readonly EventsIsEnabledDelegate EVENTS_IS_ENABLED;
+ 
+        private static readonly DeleteContextDelegate DELETE_CONTEXT;
+        
+        private static readonly ServicesWithAsyncDelegate SERVICES_WITH_ASYNC;
+        private static readonly ServicesWithServerKeepPortableDelegate SERVICES_WITH_SERVER_KEEP_PORTABLE;
+        private static readonly ServicesCancelDelegate SERVICES_CANCEL;
+        private static readonly ServicesCancelAllDelegate SERVICES_CANCEL_ALL;
+        private static readonly ServicesGetServiceProxyDelegate SERVICES_GET_SERVICE_PROXY;
+        // ReSharper restore InconsistentNaming
+
+        #endregion
+
+        /** Library pointer. */
+        private static readonly IntPtr Ptr;
+
+        /// <summary>
+        /// Initializer.
+        /// </summary>
+        [SuppressMessage("Microsoft.Design", "CA1065:DoNotRaiseExceptionsInUnexpectedLocations")]
+        static UnmanagedUtils()
+        {
+            var path = IgniteUtils.UnpackEmbeddedResource(IgniteUtils.FileIgniteJniDll);
+
+            Ptr = NativeMethods.LoadLibrary(path);
+
+            if (Ptr == IntPtr.Zero)
+                throw new IgniteException("Failed to load " + IgniteUtils.FileIgniteJniDll + ": " + Marshal.GetLastWin32Error());
+
+            REALLOCATE = CreateDelegate<ReallocateDelegate>(ProcReallocate);
+
+            IGNITION_START = CreateDelegate<IgnitionStartDelegate>(ProcIgnitionStart);
+            IGNITION_STOP = CreateDelegate<IgnitionStopDelegate>(ProcIgnitionStop);
+            IGNITION_STOP_ALL = CreateDelegate<IgnitionStopAllDelegate>(ProcIgnitionStopAll);
+            
+            PROCESSOR_RELEASE_START = CreateDelegate<ProcessorReleaseStartDelegate>(ProcProcessorReleaseStart);
+            PROCESSOR_PROJECTION = CreateDelegate<ProcessorProjectionDelegate>(ProcProcessorProjection);
+            PROCESSOR_CACHE = CreateDelegate<ProcessorCacheDelegate>(ProcProcessorCache);
+            PROCESSOR_CREATE_CACHE = CreateDelegate<ProcessorCreateCacheDelegate>(ProcProcessorCreateCache);
+            PROCESSOR_GET_OR_CREATE_CACHE = CreateDelegate<ProcessorGetOrCreateCacheDelegate>(ProcProcessorGetOrCreateCache);
+            PROCESSOR_AFFINITY = CreateDelegate<ProcessorAffinityDelegate>(ProcProcessorAffinity);
+            PROCESSOR_DATA_STREAMER = CreateDelegate<ProcessorDataStreamerDelegate>(ProcProcessorDataStreamer);
+            PROCESSOR_TRANSACTIONS = CreateDelegate<ProcessorTransactionsDelegate>(ProcProcessorTransactions);
+            PROCESSOR_COMPUTE = CreateDelegate<ProcessorComputeDelegate>(ProcProcessorCompute);
+            PROCESSOR_MESSAGE = CreateDelegate<ProcessorMessageDelegate>(ProcProcessorMessage);
+            PROCESSOR_EVENTS = CreateDelegate<ProcessorEventsDelegate>(ProcProcessorEvents);
+            PROCESSOR_SERVICES = CreateDelegate<ProcessorServicesDelegate>(ProcProcessorServices);
+            PROCESSOR_EXTENSIONS = CreateDelegate<ProcessorExtensionsDelegate>(ProcProcessorExtensions);
+            
+            TARGET_IN_STREAM_OUT_LONG = CreateDelegate<TargetInStreamOutLongDelegate>(ProcTargetInStreamOutLong);
+            TARGET_IN_STREAM_OUT_STREAM = CreateDelegate<TargetInStreamOutStreamDelegate>(ProcTargetInStreamOutStream);
+            TARGET_IN_STREAM_OUT_OBJECT = CreateDelegate<TargetInStreamOutObjectDelegate>(ProcTargetInStreamOutObject);
+            TARGET_IN_OBJECT_STREAM_OUT_STREAM = CreateDelegate<TargetInObjectStreamOutStreamDelegate>(ProcTargetInObjectStreamOutStream);
+            TARGET_OUT_LONG = CreateDelegate<TargetOutLongDelegate>(ProcTargetOutLong);
+            TARGET_OUT_STREAM = CreateDelegate<TargetOutStreamDelegate>(ProcTargetOutStream);
+            TARGET_OUT_OBJECT = CreateDelegate<TargetOutObjectDelegate>(ProcTargetOutObject);
+            TargetListenFut = CreateDelegate<TargetListenFutureDelegate>(ProcTargetListenFut);
+            TargetListenFutForOp = CreateDelegate<TargetListenFutureForOpDelegate>(ProcTargetListenFutForOp);
+
+            AffinityParts = CreateDelegate<AffinityPartitionsDelegate>(ProcAffinityParts);
+
+            CACHE_WITH_SKIP_STORE = CreateDelegate<CacheWithSkipStoreDelegate>(ProcCacheWithSkipStore);
+            CACHE_WITH_NO_RETRIES = CreateDelegate<CacheNoRetriesDelegate>(ProcCacheWithNoRetries);
+            CACHE_WITH_EXPIRY_POLICY = CreateDelegate<CacheWithExpiryPolicyDelegate>(ProcCacheWithExpiryPolicy);
+            CACHE_WITH_ASYNC = CreateDelegate<CacheWithAsyncDelegate>(ProcCacheWithAsync);
+            CACHE_WITH_KEEP_PORTABLE = CreateDelegate<CacheWithKeepPortableDelegate>(ProcCacheWithKeepPortable);
+            CACHE_CLEAR = CreateDelegate<CacheClearDelegate>(ProcCacheClear);
+            CACHE_REMOVE_ALL = CreateDelegate<CacheRemoveAllDelegate>(ProcCacheRemoveAll);
+            CACHE_OUT_OP_QUERY_CURSOR = CreateDelegate<CacheOutOpQueryCursorDelegate>(ProcCacheOutOpQueryCursor);
+            CACHE_OUT_OP_CONTINUOUS_QUERY = CreateDelegate<CacheOutOpContinuousQueryDelegate>(ProcCacheOutOpContinuousQuery);
+            CACHE_ITERATOR = CreateDelegate<CacheIteratorDelegate>(ProcCacheIterator);
+            CACHE_LOCAL_ITERATOR = CreateDelegate<CacheLocalIteratorDelegate>(ProcCacheLocalIterator);
+            CACHE_ENTER_LOCK = CreateDelegate<CacheEnterLockDelegate>(ProcCacheEnterLock);
+            CACHE_EXIT_LOCK = CreateDelegate<CacheExitLockDelegate>(ProcCacheExitLock);
+            CACHE_TRY_ENTER_LOCK = CreateDelegate<CacheTryEnterLockDelegate>(ProcCacheTryEnterLock);
+            CACHE_CLOSE_LOCK = CreateDelegate<CacheCloseLockDelegate>(ProcCacheCloseLock);
+            CACHE_REBALANCE = CreateDelegate<CacheRebalanceDelegate>(ProcCacheRebalance);
+            CACHE_SIZE = CreateDelegate<CacheSizeDelegate>(ProcCacheSize);
+
+            CACHE_STORE_CALLBACK_INVOKE = CreateDelegate<CacheStoreCallbackInvokeDelegate>(ProcCacheStoreCallbackInvoke);
+
+            COMPUTE_WITH_NO_FAILOVER = CreateDelegate<ComputeWithNoFailoverDelegate>(ProcComputeWithNoFailover);
+            COMPUTE_WITH_TIMEOUT = CreateDelegate<ComputeWithTimeoutDelegate>(ProcComputeWithTimeout);
+            COMPUTE_EXECUTE_NATIVE = CreateDelegate<ComputeExecuteNativeDelegate>(ProcComputeExecuteNative);
+
+            ContinuousQryClose = CreateDelegate<ContinuousQueryCloseDelegate>(ProcContinuousQryClose);
+            ContinuousQryGetInitialQueryCursor = CreateDelegate<ContinuousQueryGetInitialQueryCursorDelegate>(ProcContinuousQryGetInitialQueryCursor);
+
+            DataStreamerListenTop = CreateDelegate<DataStreamerListenTopologyDelegate>(ProcDataStreamerListenTop); 
+            DATA_STREAMER_ALLOW_OVERWRITE_GET = CreateDelegate<DataStreamerAllowOverwriteGetDelegate>(ProcDataStreamerAllowOverwriteGet);
+            DATA_STREAMER_ALLOW_OVERWRITE_SET = CreateDelegate<DataStreamerAllowOverwriteSetDelegate>(ProcDataStreamerAllowOverwriteSet); 
+            DATA_STREAMER_SKIP_STORE_GET = CreateDelegate<DataStreamerSkipStoreGetDelegate>(ProcDataStreamerSkipStoreGet); 
+            DATA_STREAMER_SKIP_STORE_SET = CreateDelegate<DataStreamerSkipStoreSetDelegate>(ProcDataStreamerSkipStoreSet); 
+            DATA_STREAMER_PER_NODE_BUFFER_SIZE_GET = CreateDelegate<DataStreamerPerNodeBufferSizeGetDelegate>(ProcDataStreamerPerNodeBufferSizeGet); 
+            DATA_STREAMER_PER_NODE_BUFFER_SIZE_SET = CreateDelegate<DataStreamerPerNodeBufferSizeSetDelegate>(ProcDataStreamerPerNodeBufferSizeSet); 
+            DataStreamerPerNodeParallelOpsGet = CreateDelegate<DataStreamerPerNodeParallelOperationsGetDelegate>(ProcDataStreamerPerNodeParallelOpsGet); 
+            DataStreamerPerNodeParallelOpsSet = CreateDelegate<DataStreamerPerNodeParallelOperationsSetDelegate>(ProcDataStreamerPerNodeParallelOpsSet); 
+
+            MessagingWithAsync = CreateDelegate<MessagingWithAsyncDelegate>(ProcMessagingWithAsync);
+
+            PROJECTION_FOR_OTHERS = CreateDelegate<ProjectionForOthersDelegate>(ProcProjectionForOthers);
+            PROJECTION_FOR_REMOTES = CreateDelegate<ProjectionForRemotesDelegate>(ProcProjectionForRemotes);
+            PROJECTION_FOR_DAEMONS = CreateDelegate<ProjectionForDaemonsDelegate>(ProcProjectionForDaemons);
+            PROJECTION_FOR_RANDOM = CreateDelegate<ProjectionForRandomDelegate>(ProcProjectionForRandom);
+            PROJECTION_FOR_OLDEST = CreateDelegate<ProjectionForOldestDelegate>(ProcProjectionForOldest);
+            PROJECTION_FOR_YOUNGEST = CreateDelegate<ProjectionForYoungestDelegate>(ProcProjectionForYoungest);
+            PROJECTION_RESET_METRICS = CreateDelegate<ProjectionResetMetricsDelegate>(ProcProjectionResetMetrics);
+            PROJECTION_OUT_OP_RET = CreateDelegate<ProjectionOutOpRetDelegate>(ProcProjectionOutOpRet);
+
+            QryCursorIterator = CreateDelegate<QueryCursorIteratorDelegate>(ProcQryCursorIterator);
+            QryCursorClose = CreateDelegate<QueryCursorCloseDelegate>(ProcQryCursorClose);
+
+            RELEASE = CreateDelegate<ReleaseDelegate>(ProcRelease);
+
+            TxStart = CreateDelegate<TransactionsStartDelegate>(ProcTxStart);
+            TxCommit = CreateDelegate<TransactionsCommitDelegate>(ProcTxCommit);
+            TxCommitAsync = CreateDelegate<TransactionsCommitAsyncDelegate>(ProcTxCommitAsync);
+            TxRollback = CreateDelegate<TransactionsRollbackDelegate>(ProcTxRollback);
+            TxRollbackAsync = CreateDelegate<TransactionsRollbackAsyncDelegate>(ProcTxRollbackAsync);
+            TxClose = CreateDelegate<TransactionsCloseDelegate>(ProcTxClose);
+            TxState = CreateDelegate<TransactionsStateDelegate>(ProcTxState);
+            TxSetRollbackOnly = CreateDelegate<TransactionsSetRollbackOnlyDelegate>(ProcTxSetRollbackOnly);
+            TxResetMetrics = CreateDelegate<TransactionsResetMetricsDelegate>(ProcTxResetMetrics);
+
+            THROW_TO_JAVA = CreateDelegate<ThrowToJavaDelegate>(ProcThrowToJava);
+
+            HANDLERS_SIZE = CreateDelegate<HandlersSizeDelegate>(ProcHandlersSize);
+
+            CREATE_CONTEXT = CreateDelegate<CreateContextDelegate>(ProcCreateContext);
+            DELETE_CONTEXT = CreateDelegate<DeleteContextDelegate>(ProcDeleteContext);
+
+            DESTROY_JVM = CreateDelegate<DestroyJvmDelegate>(ProcDestroyJvm);
+
+            EVENTS_WITH_ASYNC = CreateDelegate<EventsWithAsyncDelegate>(ProcEventsWithAsync);
+            EVENTS_STOP_LOCAL_LISTEN = CreateDelegate<EventsStopLocalListenDelegate>(ProcEventsStopLocalListen);
+            EVENTS_LOCAL_LISTEN = CreateDelegate<EventsLocalListenDelegate>(ProcEventsLocalListen);
+            EVENTS_IS_ENABLED = CreateDelegate<EventsIsEnabledDelegate>(ProcEventsIsEnabled);
+            
+            SERVICES_WITH_ASYNC = CreateDelegate<ServicesWithAsyncDelegate>(ProcServicesWithAsync);
+            SERVICES_WITH_SERVER_KEEP_PORTABLE = CreateDelegate<ServicesWithServerKeepPortableDelegate>(ProcServicesWithServerKeepPortable);
+            SERVICES_CANCEL = CreateDelegate<ServicesCancelDelegate>(ProcServicesCancel);
+            SERVICES_CANCEL_ALL = CreateDelegate<ServicesCancelAllDelegate>(ProcServicesCancelAll);
+            SERVICES_GET_SERVICE_PROXY = CreateDelegate<ServicesGetServiceProxyDelegate>(ProcServicesGetServiceProxy);
+        }
+
+        #region NATIVE METHODS: PROCESSOR
+
+        internal static IUnmanagedTarget IgnitionStart(UnmanagedContext ctx, string cfgPath, string gridName, 
+            bool clientMode)
+        {
+            using (var mem = IgniteManager.Memory.Allocate().Stream())
+            {
+                mem.WriteBool(clientMode);
+
+                sbyte* cfgPath0 = IgniteUtils.StringToUtf8Unmanaged(cfgPath);
+                sbyte* gridName0 = IgniteUtils.StringToUtf8Unmanaged(gridName);
+
+                try
+                {
+                    void* res = IGNITION_START(ctx.NativeContext, cfgPath0, gridName0, InteropFactoryId,
+                        mem.SynchronizeOutput());
+
+                    return new UnmanagedTarget(ctx, res);
+                }
+                finally
+                {
+                    Marshal.FreeHGlobal(new IntPtr(cfgPath0));
+                    Marshal.FreeHGlobal(new IntPtr(gridName0));
+                }
+            }
+        }
+
+        internal static bool IgnitionStop(void* ctx, string gridName, bool cancel)
+        {
+            sbyte* gridName0 = IgniteUtils.StringToUtf8Unmanaged(gridName);
+
+            try
+            {
+                return IGNITION_STOP(ctx, gridName0, cancel);
+            }
+            finally
+            {
+                Marshal.FreeHGlobal(new IntPtr(gridName0));
+            }
+        }
+
+        internal static void IgnitionStopAll(void* ctx, bool cancel)
+        {
+            IGNITION_STOP_ALL(ctx, cancel);
+        }
+        
+        internal static void ProcessorReleaseStart(IUnmanagedTarget target)
+        {
+            PROCESSOR_RELEASE_START(target.Context, target.Target);
+        }
+
+        internal static IUnmanagedTarget ProcessorProjection(IUnmanagedTarget target)
+        {
+            void* res = PROCESSOR_PROJECTION(target.Context, target.Target);
+
+            return target.ChangeTarget(res);
+        }
+
+        internal static IUnmanagedTarget ProcessorCache(IUnmanagedTarget target, string name)
+        {
+            sbyte* name0 = IgniteUtils.StringToUtf8Unmanaged(name);
+
+            try
+            {
+                void* res = PROCESSOR_CACHE(target.Context, target.Target, name0);
+
+                return target.ChangeTarget(res);
+            }
+            finally
+            {
+                Marshal.FreeHGlobal(new IntPtr(name0));
+            }
+        }
+
+        internal static IUnmanagedTarget ProcessorCreateCache(IUnmanagedTarget target, string name)
+        {
+            sbyte* name0 = IgniteUtils.StringToUtf8Unmanaged(name);
+
+            try
+            {
+                void* res = PROCESSOR_CREATE_CACHE(target.Context, target.Target, name0);
+
+                return target.ChangeTarget(res);
+            }
+            finally
+            {
+                Marshal.FreeHGlobal(new IntPtr(name0));
+            }
+        }
+
+        internal static IUnmanagedTarget ProcessorGetOrCreateCache(IUnmanagedTarget target, string name)
+        {
+            sbyte* name0 = IgniteUtils.StringToUtf8Unmanaged(name);
+
+            try
+            {
+                void* res = PROCESSOR_GET_OR_CREATE_CACHE(target.Context, target.Target, name0);
+
+                return target.ChangeTarget(res);
+            }
+            finally
+            {
+                Marshal.FreeHGlobal(new IntPtr(name0));
+            }
+        }
+
+        internal static IUnmanagedTarget ProcessorAffinity(IUnmanagedTarget target, string name)
+        {
+            sbyte* name0 = IgniteUtils.StringToUtf8Unmanaged(name);
+
+            try
+            {
+                void* res = PROCESSOR_AFFINITY(target.Context, target.Target, name0);
+
+                return target.ChangeTarget(res);
+            }
+            finally
+            {
+                Marshal.FreeHGlobal(new IntPtr(name0));
+            }
+        }
+
+        internal static IUnmanagedTarget ProcessorDataStreamer(IUnmanagedTarget target, string name, bool keepPortable)
+        {
+            sbyte* name0 = IgniteUtils.StringToUtf8Unmanaged(name);
+
+            try
+            {
+                void* res = PROCESSOR_DATA_STREAMER(target.Context, target.Target, name0, keepPortable);
+
+                return target.ChangeTarget(res);
+            }
+            finally
+            {
+                Marshal.FreeHGlobal(new IntPtr(name0));
+            }
+        }
+        
+        internal static IUnmanagedTarget ProcessorTransactions(IUnmanagedTarget target)
+        {
+            void* res = PROCESSOR_TRANSACTIONS(target.Context, target.Target);
+
+            return target.ChangeTarget(res);
+        }
+
+        internal static IUnmanagedTarget ProcessorCompute(IUnmanagedTarget target, IUnmanagedTarget prj)
+        {
+            void* res = PROCESSOR_COMPUTE(target.Context, target.Target, prj.Target);
+
+            return target.ChangeTarget(res);
+        }
+
+        internal static IUnmanagedTarget ProcessorMessage(IUnmanagedTarget target, IUnmanagedTarget prj)
+        {
+            void* res = PROCESSOR_MESSAGE(target.Context, target.Target, prj.Target);
+
+            return target.ChangeTarget(res);
+        }
+
+        internal static IUnmanagedTarget ProcessorEvents(IUnmanagedTarget target, IUnmanagedTarget prj)
+        {
+            void* res = PROCESSOR_EVENTS(target.Context, target.Target, prj.Target);
+
+            return target.ChangeTarget(res);
+        }
+
+        internal static IUnmanagedTarget ProcessorServices(IUnmanagedTarget target, IUnmanagedTarget prj)
+        {
+            void* res = PROCESSOR_SERVICES(target.Context, target.Target, prj.Target);
+
+            return target.ChangeTarget(res);
+        }
+
+        internal static IUnmanagedTarget ProcessorExtensions(IUnmanagedTarget target)
+        {
+            void* res = PROCESSOR_EXTENSIONS(target.Context, target.Target);
+
+            return target.ChangeTarget(res);
+        }
+
+        #endregion
+
+        #region NATIVE METHODS: TARGET
+
+        internal static long TargetInStreamOutLong(IUnmanagedTarget target, int opType, long memPtr)
+        {
+            return TARGET_IN_STREAM_OUT_LONG(target.Context, target.Target, opType, memPtr);
+        }
+
+        internal static void TargetInStreamOutStream(IUnmanagedTarget target, int opType, long inMemPtr, long outMemPtr)
+        {
+            TARGET_IN_STREAM_OUT_STREAM(target.Context, target.Target, opType, inMemPtr, outMemPtr);
+        }
+
+        internal static IUnmanagedTarget TargetInStreamOutObject(IUnmanagedTarget target, int opType, long inMemPtr)
+        {
+            void* res = TARGET_IN_STREAM_OUT_OBJECT(target.Context, target.Target, opType, inMemPtr);
+
+            return target.ChangeTarget(res);
+        }
+
+        internal static void TargetInObjectStreamOutStream(IUnmanagedTarget target, int opType, void* arg, long inMemPtr, long outMemPtr)
+        {
+            TARGET_IN_OBJECT_STREAM_OUT_STREAM(target.Context, target.Target, opType, arg, inMemPtr, outMemPtr);
+        }
+
+        internal static long TargetOutLong(IUnmanagedTarget target, int opType)
+        {
+            return TARGET_OUT_LONG(target.Context, target.Target, opType);
+        }
+
+        internal static void TargetOutStream(IUnmanagedTarget target, int opType, long memPtr)
+        {
+            TARGET_OUT_STREAM(target.Context, target.Target, opType, memPtr);
+        }
+
+        internal static IUnmanagedTarget TargetOutObject(IUnmanagedTarget target, int opType)
+        {
+            void* res = TARGET_OUT_OBJECT(target.Context, target.Target, opType);
+
+            return target.ChangeTarget(res);
+        }
+
+        internal static void TargetListenFuture(IUnmanagedTarget target, long futId, int typ)
+        {
+            TargetListenFut(target.Context, target.Target, futId, typ);
+        }
+
+        internal static void TargetListenFutureForOperation(IUnmanagedTarget target, long futId, int typ, int opId)
+        {
+            TargetListenFutForOp(target.Context, target.Target, futId, typ, opId);
+        }
+
+        #endregion
+
+        #region NATIVE METHODS: AFFINITY
+
+        internal static int AffinityPartitions(IUnmanagedTarget target)
+        {
+            return AffinityParts(target.Context, target.Target);
+        }
+
+        #endregion
+
+        #region NATIVE METHODS: CACHE
+
+        internal static IUnmanagedTarget CacheWithSkipStore(IUnmanagedTarget target)
+        {
+            void* res = CACHE_WITH_SKIP_STORE(target.Context, target.Target);
+
+            return target.ChangeTarget(res);
+        }
+
+        internal static IUnmanagedTarget CacheWithNoRetries(IUnmanagedTarget target)
+        {
+            void* res = CACHE_WITH_NO_RETRIES(target.Context, target.Target);
+
+            return target.ChangeTarget(res);
+        }
+
+        internal static IUnmanagedTarget CacheWithExpiryPolicy(IUnmanagedTarget target, long create, long update, long access)
+        {
+            void* res = CACHE_WITH_EXPIRY_POLICY(target.Context, target.Target, create, update, access);
+
+            return target.ChangeTarget(res);
+        }
+
+        internal static IUnmanagedTarget CacheWithAsync(IUnmanagedTarget target)
+        {
+            void* res = CACHE_WITH_ASYNC(target.Context, target.Target);
+
+            return target.ChangeTarget(res);
+        }
+
+        internal static IUnmanagedTarget CacheWithKeepPortable(IUnmanagedTarget target)
+        {
+            void* res = CACHE_WITH_KEEP_PORTABLE(target.Context, target.Target);
+
+            return target.ChangeTarget(res);
+        }
+
+        internal static void CacheClear(IUnmanagedTarget target)
+        {
+            CACHE_CLEAR(target.Context, target.Target);
+        }
+
+        internal static void CacheRemoveAll(IUnmanagedTarget target)
+        {
+            CACHE_REMOVE_ALL(target.Context, target.Target);
+        }
+
+        internal static IUnmanagedTarget CacheOutOpQueryCursor(IUnmanagedTarget target, int type, long memPtr)
+        {
+            void* res = CACHE_OUT_OP_QUERY_CURSOR(target.Context, target.Target, type, memPtr);
+
+            return target.ChangeTarget(res);
+        }
+
+        internal static IUnmanagedTarget CacheOutOpContinuousQuery(IUnmanagedTarget target, int type, long memPtr)
+        {
+            void* res = CACHE_OUT_OP_CONTINUOUS_QUERY(target.Context, target.Target, type, memPtr);
+
+            return target.ChangeTarget(res);
+        }
+
+        internal static IUnmanagedTarget CacheIterator(IUnmanagedTarget target)
+        {
+            void* res = CACHE_ITERATOR(target.Context, target.Target);
+
+            return target.ChangeTarget(res);
+        }
+
+        internal static IUnmanagedTarget CacheLocalIterator(IUnmanagedTarget target, int peekModes)
+        {
+            void* res = CACHE_LOCAL_ITERATOR(target.Context, target.Target, peekModes);
+
+            return target.ChangeTarget(res);
+        }
+
+        internal static void CacheEnterLock(IUnmanagedTarget target, long id)
+        {
+            CACHE_ENTER_LOCK(target.Context, target.Target, id);
+        }
+
+        internal static void CacheExitLock(IUnmanagedTarget target, long id)
+        {
+            CACHE_EXIT_LOCK(target.Context, target.Target, id);
+        }
+
+        internal static bool CacheTryEnterLock(IUnmanagedTarget target, long id, long timeout)
+        {
+            return CACHE_TRY_ENTER_LOCK(target.Context, target.Target, id, timeout);
+        }
+
+        internal static void CacheCloseLock(IUnmanagedTarget target, long id)
+        {
+            CACHE_CLOSE_LOCK(target.Context, target.Target, id);
+        }
+
+        internal static void CacheRebalance(IUnmanagedTarget target, long futId)
+        {
+            CACHE_REBALANCE(target.Context, target.Target, futId);
+        }
+
+        internal static void CacheStoreCallbackInvoke(IUnmanagedTarget target, long memPtr)
+        {
+            CACHE_STORE_CALLBACK_INVOKE(target.Context, target.Target, memPtr);
+        }
+
+        internal static int CacheSize(IUnmanagedTarget target, int modes, bool loc)
+        {
+            return CACHE_SIZE(target.Context, target.Target, modes, loc);
+        }
+
+        #endregion
+
+        #region NATIVE METHODS: COMPUTE
+
+        internal static void ComputeWithNoFailover(IUnmanagedTarget target)
+        {
+            COMPUTE_WITH_NO_FAILOVER(target.Context, target.Target);
+        }
+
+        internal static void ComputeWithTimeout(IUnmanagedTarget target, long timeout)
+        {
+            COMPUTE_WITH_TIMEOUT(target.Context, target.Target, timeout);
+        }
+
+        internal static void ComputeExecuteNative(IUnmanagedTarget target, long taskPtr, long topVer)
+        {
+            COMPUTE_EXECUTE_NATIVE(target.Context, target.Target, taskPtr, topVer);
+        }
+
+        #endregion
+
+        #region NATIVE METHODS: CONTINUOUS QUERY
+
+        internal static void ContinuousQueryClose(IUnmanagedTarget target)
+        {
+            ContinuousQryClose(target.Context, target.Target);
+        }
+
+        internal static IUnmanagedTarget ContinuousQueryGetInitialQueryCursor(IUnmanagedTarget target)
+        {
+            void* res = ContinuousQryGetInitialQueryCursor(target.Context, target.Target);
+
+            return res == null ? null : target.ChangeTarget(res);
+        }
+
+        #endregion
+
+        #region NATIVE METHODS: DATA STREAMER
+
+        internal static void DataStreamerListenTopology(IUnmanagedTarget target, long ptr)
+        {
+            DataStreamerListenTop(target.Context, target.Target, ptr);
+        }
+
+        internal static bool DataStreamerAllowOverwriteGet(IUnmanagedTarget target)
+        {
+            return DATA_STREAMER_ALLOW_OVERWRITE_GET(target.Context, target.Target);
+        }
+
+        internal static void DataStreamerAllowOverwriteSet(IUnmanagedTarget target, bool val)
+        {
+            DATA_STREAMER_ALLOW_OVERWRITE_SET(target.Context, target.Target, val);
+        }
+
+        internal static bool DataStreamerSkipStoreGet(IUnmanagedTarget target)
+        {
+            return DATA_STREAMER_SKIP_STORE_GET(target.Context, target.Target);
+        }
+
+        internal static void DataStreamerSkipStoreSet(IUnmanagedTarget target, bool val)
+        {
+            DATA_STREAMER_SKIP_STORE_SET(target.Context, target.Target, val);
+        }
+
+        internal static int DataStreamerPerNodeBufferSizeGet(IUnmanagedTarget target)
+        {
+            return DATA_STREAMER_PER_NODE_BUFFER_SIZE_GET(target.Context, target.Target);
+        }
+
+        internal static void DataStreamerPerNodeBufferSizeSet(IUnmanagedTarget target, int val)
+        {
+            DATA_STREAMER_PER_NODE_BUFFER_SIZE_SET(target.Context, target.Target, val);
+        }
+
+        internal static int DataStreamerPerNodeParallelOperationsGet(IUnmanagedTarget target)
+        {
+            return DataStreamerPerNodeParallelOpsGet(target.Context, target.Target);
+        }
+
+        internal static void DataStreamerPerNodeParallelOperationsSet(IUnmanagedTarget target, int val)
+        {
+            DataStreamerPerNodeParallelOpsSet(target.Context, target.Target, val);
+        }
+
+        #endregion
+
+        #region NATIVE METHODS: MESSAGING
+
+        internal static IUnmanagedTarget MessagingWithASync(IUnmanagedTarget target)
+        {
+            void* res = MessagingWithAsync(target.Context, target.Target);
+
+            return target.ChangeTarget(res);
+        }
+
+        #endregion
+
+        #region NATIVE METHODS: PROJECTION
+
+        internal static IUnmanagedTarget ProjectionForOthers(IUnmanagedTarget target, IUnmanagedTarget prj)
+        {
+            void* res = PROJECTION_FOR_OTHERS(target.Context, target.Target, prj.Target);
+
+            return target.ChangeTarget(res);
+        }
+
+        internal static IUnmanagedTarget ProjectionForRemotes(IUnmanagedTarget target)
+        {
+            void* res = PROJECTION_FOR_REMOTES(target.Context, target.Target);
+
+            return target.ChangeTarget(res);
+        }
+
+        internal static IUnmanagedTarget ProjectionForDaemons(IUnmanagedTarget target)
+        {
+            void* res = PROJECTION_FOR_DAEMONS(target.Context, target.Target);
+
+            return target.ChangeTarget(res);
+        }
+
+        internal static IUnmanagedTarget ProjectionForRandom(IUnmanagedTarget target)
+        {
+            void* res = PROJECTION_FOR_RANDOM(target.Context, target.Target);
+
+            return target.ChangeTarget(res);
+        }
+
+        internal static IUnmanagedTarget ProjectionForOldest(IUnmanagedTarget target)
+        {
+            void* res = PROJECTION_FOR_OLDEST(target.Context, target.Target);
+
+            return target.ChangeTarget(res);
+        }
+
+        internal static IUnmanagedTarget ProjectionForYoungest(IUnmanagedTarget target)
+        {
+            void* res = PROJECTION_FOR_YOUNGEST(target.Context, target.Target);
+
+            return target.ChangeTarget(res);
+        }
+        
+        internal static void ProjectionResetMetrics(IUnmanagedTarget target)
+        {
+            PROJECTION_RESET_METRICS(target.Context, target.Target);
+        }
+
+        internal static IUnmanagedTarget ProjectionOutOpRet(IUnmanagedTarget target, int type, long memPtr)
+        {
+            void* res = PROJECTION_OUT_OP_RET(target.Context, target.Target, type, memPtr);
+
+            return target.ChangeTarget(res);
+        }
+
+        #endregion
+
+        #region NATIVE METHODS: QUERY CURSOR
+
+        internal static void QueryCursorIterator(IUnmanagedTarget target)
+        {
+            QryCursorIterator(target.Context, target.Target);
+        }
+
+        internal static void QueryCursorClose(IUnmanagedTarget target)
+        {
+            QryCursorClose(target.Context, target.Target);
+        }
+
+        #endregion
+
+        #region NATIVE METHODS: TRANSACTIONS
+
+        internal static long TransactionsStart(IUnmanagedTarget target, int concurrency, int isolation, long timeout, int txSize)
+        {
+            return TxStart(target.Context, target.Target, concurrency, isolation, timeout, txSize);
+        }
+
+        internal static int TransactionsCommit(IUnmanagedTarget target, long id)
+        {
+            return TxCommit(target.Context, target.Target, id);
+        }
+
+        internal static void TransactionsCommitAsync(IUnmanagedTarget target, long id, long futId)
+        {
+            TxCommitAsync(target.Context, target.Target, id, futId);
+        }
+
+        internal static int TransactionsRollback(IUnmanagedTarget target, long id)
+        {
+            return TxRollback(target.Context, target.Target, id);
+        }
+
+        internal static void TransactionsRollbackAsync(IUnmanagedTarget target, long id, long futId)
+        {
+            TxRollbackAsync(target.Context, target.Target, id, futId);
+        }
+
+        internal static int TransactionsClose(IUnmanagedTarget target, long id)
+        {
+            return TxClose(target.Context, target.Target, id);
+        }
+
+        internal static int TransactionsState(IUnmanagedTarget target, long id)
+        {
+            return TxState(target.Context, target.Target, id);
+        }
+
+        internal static bool TransactionsSetRollbackOnly(IUnmanagedTarget target, long id)
+        {
+            return TxSetRollbackOnly(target.Context, target.Target, id);
+        }
+
+        internal static void TransactionsResetMetrics(IUnmanagedTarget target)
+        {
+            TxResetMetrics(target.Context, target.Target);
+        }
+
+        #endregion
+
+        #region NATIVE METHODS: MISCELANNEOUS
+
+        internal static void Reallocate(long memPtr, int cap)
+        {
+            int res = REALLOCATE(memPtr, cap);
+
+            if (res != 0)
+                throw new IgniteException("Failed to reallocate external memory [ptr=" + memPtr + 
+                    ", capacity=" + cap + ']');
+        }
+
+        internal static void Release(IUnmanagedTarget target)
+        {
+            RELEASE(target.Target);
+        }
+
+        internal static void ThrowToJava(void* ctx, Exception e)
+        {
+            char* msgChars = (char*)IgniteUtils.StringToUtf8Unmanaged(e.Message);
+
+            try
+            {
+                THROW_TO_JAVA(ctx, msgChars);
+            }
+            finally
+            {
+                Marshal.FreeHGlobal(new IntPtr(msgChars));
+            }
+        }
+
+        
+
+        internal static int HandlersSize()
+        {
+            return HANDLERS_SIZE();
+        }
+
+        internal static void* CreateContext(void* opts, int optsLen, void* cbs)
+        {
+            return CREATE_CONTEXT(opts, optsLen, cbs);
+        }
+
+        internal static void DeleteContext(void* ctx)
+        {
+            DELETE_CONTEXT(ctx);
+        }
+
+        internal static void DestroyJvm(void* ctx)
+        {
+            DESTROY_JVM(ctx);
+        }
+
+        #endregion
+
+        #region NATIVE METHODS: EVENTS
+
+        internal static IUnmanagedTarget EventsWithAsync(IUnmanagedTarget target)
+        {
+            return target.ChangeTarget(EVENTS_WITH_ASYNC(target.Context, target.Target));
+        }
+
+        internal static bool EventsStopLocalListen(IUnmanagedTarget target, long handle)
+        {
+            return EVENTS_STOP_LOCAL_LISTEN(target.Context, target.Target, handle);
+        }
+
+        internal static bool EventsIsEnabled(IUnmanagedTarget target, int type)
+        {
+            return EVENTS_IS_ENABLED(target.Context, target.Target, type);
+        }
+
+        internal static void EventsLocalListen(IUnmanagedTarget target, long handle, int type)
+        {
+            EVENTS_LOCAL_LISTEN(target.Context, target.Target, handle, type);
+        }
+
+        #endregion
+
+        #region NATIVE METHODS: SERVICES
+
+        internal static IUnmanagedTarget ServicesWithAsync(IUnmanagedTarget target)
+        {
+            return target.ChangeTarget(SERVICES_WITH_ASYNC(target.Context, target.Target));
+        }
+
+        internal static IUnmanagedTarget ServicesWithServerKeepPortable(IUnmanagedTarget target)
+        {
+            return target.ChangeTarget(SERVICES_WITH_SERVER_KEEP_PORTABLE(target.Context, target.Target));
+        }
+
+        internal static void ServicesCancel(IUnmanagedTarget target, string name)
+        {
+            var nameChars = (char*)IgniteUtils.StringToUtf8Unmanaged(name);
+
+            try
+            {
+                SERVICES_CANCEL(target.Context, target.Target, nameChars);
+            }
+            finally
+            {
+                Marshal.FreeHGlobal(new IntPtr(nameChars));
+            }
+        }
+
+        internal static void ServicesCancelAll(IUnmanagedTarget target)
+        {
+            SERVICES_CANCEL_ALL(target.Context, target.Target);
+        }
+
+        internal static IUnmanagedTarget ServicesGetServiceProxy(IUnmanagedTarget target, string name, bool sticky)
+        {
+            var nameChars = (char*)IgniteUtils.StringToUtf8Unmanaged(name);
+
+            try
+            {
+                return target.ChangeTarget(
+                    SERVICES_GET_SERVICE_PROXY(target.Context, target.Target, nameChars, sticky));
+            }
+            finally
+            {
+                Marshal.FreeHGlobal(new IntPtr(nameChars));
+            }
+        }
+
+        #endregion
+
+        /// <summary>
+        /// No-op initializer used to force type loading and static constructor call.
+        /// </summary>
+        internal static void Initialize()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Create delegate for the given procedure.
+        /// </summary>
+        /// <typeparam name="T">Delegate type.</typeparam>
+        /// <param name="procName">Procedure name.</param>
+        /// <returns></returns>
+        private static T CreateDelegate<T>(string procName)
+        {
+            var procPtr = NativeMethods.GetProcAddress(Ptr, procName);
+
+            if (procPtr == IntPtr.Zero)
+                throw new IgniteException(string.Format("Unable to find native function: {0} (Error code: {1}). " +
+                                                      "Make sure that module.def is up to date",
+                    procName, Marshal.GetLastWin32Error()));
+
+            return TypeCaster<T>.Cast(Marshal.GetDelegateForFunctionPointer(procPtr, typeof (T)));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Lifecycle/ILifecycleBean.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Lifecycle/ILifecycleBean.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Lifecycle/ILifecycleBean.cs
new file mode 100644
index 0000000..06cb523
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Lifecycle/ILifecycleBean.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.Lifecycle
+{
+    using Apache.Ignite.Core.Resource;
+
+    /// <summary>
+    /// A bean that reacts to Ignite lifecycle events defined in <see cref="LifecycleEventType"/>.
+    /// Use this bean whenever you need to plug some custom logic before or after
+    /// Ignite startup and stopping routines.
+    /// <para />
+    /// There are four events you can react to:
+    /// <list type="bullet">
+    ///     <item>
+    ///         <term>BeforeNodeStart</term>
+    ///         <description>Invoked before Ignite startup routine is initiated. Note that Ignite 
+    ///         is not available during this event, therefore if you injected an Ignite instance 
+    ///         via <see cref="InstanceResourceAttribute"/> attribute, you cannot 
+    ///         use it yet.</description>
+    ///     </item>
+    ///     <item>
+    ///         <term>AfterNodeStart</term>
+    ///         <description>Invoked right after Ignite has started. At this point, if you injected
+    ///         an Ignite instance via <see cref="InstanceResourceAttribute"/> attribute, 
+    ///         you can start using it.</description>
+    ///     </item>
+    ///     <item>
+    ///         <term>BeforeNodeStop</term>
+    ///         <description>Invoked right before Ignite stop routine is initiated. Ignite is still 
+    ///         available at this stage, so if you injected an Ignite instance via 
+    ///         <see cref="InstanceResourceAttribute"/> attribute, you can use it.
+    ///         </description>
+    ///     </item>
+    ///     <item>
+    ///         <term>AfterNodeStop</term>
+    ///         <description>Invoked right after Ignite has stopped. Note that Ignite is not available 
+    ///         during this event.</description>
+    ///     </item>
+    /// </list>
+    /// </summary>
+    public interface ILifecycleBean
+    {
+        /// <summary>
+        /// This method is called when lifecycle event occurs.
+        /// </summary>
+        /// <param name="evt">Lifecycle event.</param>
+        void OnLifecycleEvent(LifecycleEventType evt);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Lifecycle/LifecycleEventType.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Lifecycle/LifecycleEventType.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Lifecycle/LifecycleEventType.cs
new file mode 100644
index 0000000..beea555
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Lifecycle/LifecycleEventType.cs
@@ -0,0 +1,49 @@
+/*
+ * 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.Lifecycle
+{
+    /// <summary>
+    /// Ignite lifecycle event types. These events are used to notify lifecycle beans
+    /// about changes in Ignite lifecycle state.
+    /// <para />
+    /// For more information and detailed examples refer to <see cref="ILifecycleBean"/>
+    /// documentation.
+    /// </summary>
+    public enum LifecycleEventType
+    {
+        /// <summary>
+        /// Invoked before node startup routine. Node is not initialized and cannot be used.
+        /// </summary>
+        BeforeNodeStart,
+
+        /// <summary>
+        /// Invoked after node startup is complete. Node is fully initialized and fully functional.
+        /// </summary>
+        AfterNodeStart,
+
+        /// <summary>
+        /// Invoked before node stopping routine. Node is fully functional at this point.
+        /// </summary>
+        BeforeNodeStop,
+
+        /// <summary>
+        /// Invoked after node had stopped. Node is stopped and cannot be used. 
+        /// </summary>
+        AfterNodeStop
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Messaging/IMessageFilter.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Messaging/IMessageFilter.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Messaging/IMessageFilter.cs
new file mode 100644
index 0000000..456c5e6
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Messaging/IMessageFilter.cs
@@ -0,0 +1,35 @@
+/*
+ * 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.Messaging
+{
+    using System;
+
+    /// <summary>
+    /// Represents messaging filter predicate.
+    /// </summary>
+    public interface IMessageFilter<in T>
+    {
+        /// <summary>
+        /// Returns a value indicating whether provided message and node id satisfy this predicate.
+        /// </summary>
+        /// <param name="nodeId">Node identifier.</param>
+        /// <param name="message">Message.</param>
+        /// <returns>Value indicating whether provided message and node id satisfy this predicate.</returns>
+        bool Invoke(Guid nodeId, T message);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Messaging/IMessaging.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Messaging/IMessaging.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Messaging/IMessaging.cs
new file mode 100644
index 0000000..96f46b9
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Messaging/IMessaging.cs
@@ -0,0 +1,105 @@
+/*
+ * 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.Messaging
+{
+    using System;
+    using System.Collections;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// Provides functionality for topic-based message exchange among nodes defined by <see cref="IClusterGroup"/>.
+    /// Users can send ordered and unordered messages to various topics. Note that same topic name
+    /// cannot be reused between ordered and unordered messages.
+    /// <para/>
+    /// All members are thread-safe and may be used concurrently from multiple threads.
+    /// </summary>
+    public interface IMessaging : IAsyncSupport<IMessaging>
+    {
+        /// <summary>
+        /// Gets the cluster group to which this instance belongs.
+        /// </summary>
+        IClusterGroup ClusterGroup { get; }
+
+        /// <summary>
+        /// Sends a message with specified topic to the nodes in the underlying cluster group.
+        /// </summary>
+        /// <param name="message">Message to send.</param>
+        /// <param name="topic">Topic to send to, null for default topic.</param>
+        void Send(object message, object topic = null);
+
+        /// <summary>
+        /// Sends messages with specified topic to the nodes in the underlying cluster group.
+        /// </summary>
+        /// <param name="messages">Messages to send.</param>
+        /// <param name="topic">Topic to send to, null for default topic.</param>
+        void Send(IEnumerable messages, object topic = null);
+
+        /// <summary>
+        /// Sends a message with specified topic to the nodes in the underlying cluster group.
+        /// Messages sent with this method will arrive in the same order they were sent. Note that if a topic is used
+        /// for ordered messages, then it cannot be reused for non-ordered messages.
+        /// </summary>
+        /// <param name="message">Message to send.</param>
+        /// <param name="topic">Topic to send to, null for default topic.</param>
+        /// <param name="timeout">
+        /// Message timeout, null for for default value from configuration (IgniteConfiguration.getNetworkTimeout).
+        /// </param>
+        void SendOrdered(object message, object topic = null, TimeSpan? timeout = null);
+
+        /// <summary>
+        /// Adds local listener for given topic on local node only. This listener will be notified whenever any
+        /// node within the cluster group will send a message for a given topic to this node. Local listen
+        /// subscription will happen regardless of whether local node belongs to this cluster group or not.
+        /// </summary>
+        /// <param name="filter">
+        /// Predicate that is called on each received message. If predicate returns false,
+        /// then it will be unsubscribed from any further notifications.
+        /// </param>
+        /// <param name="topic">Topic to subscribe to.</param>
+        void LocalListen<T>(IMessageFilter<T> filter, object topic = null);
+
+        /// <summary>
+        /// Unregisters local listener for given topic on local node only.
+        /// </summary>
+        /// <param name="filter">Listener predicate.</param>
+        /// <param name="topic">Topic to unsubscribe from.</param>
+        void StopLocalListen<T>(IMessageFilter<T> filter, object topic = null);
+
+        /// <summary>
+        /// Adds a message listener for a given topic to all nodes in the cluster group (possibly including
+        /// this node if it belongs to the cluster group as well). This means that any node within this cluster
+        /// group can send a message for a given topic and all nodes within the cluster group will receive
+        /// listener notifications.
+        /// </summary>
+        /// <param name="filter">Listener predicate.</param>
+        /// <param name="topic">Topic to unsubscribe from.</param>
+        /// <returns>
+        /// Operation ID that can be passed to <see cref="StopRemoteListen"/> method to stop listening.
+        /// </returns>
+        [AsyncSupported]
+        Guid RemoteListen<T>(IMessageFilter<T> filter, object topic = null);
+
+        /// <summary>
+        /// Unregisters all listeners identified with provided operation ID on all nodes in the cluster group.
+        /// </summary>
+        /// <param name="opId">Operation ID that was returned from <see cref="RemoteListen{T}"/> method.</param>
+        [AsyncSupported]
+        void StopRemoteListen(Guid opId);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableBuilder.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableBuilder.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableBuilder.cs
new file mode 100644
index 0000000..ae3ab6a
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableBuilder.cs
@@ -0,0 +1,78 @@
+/*
+ * 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.Portable
+{
+    using System.Diagnostics.CodeAnalysis;
+
+    /// <summary>
+    /// Portable object builder. Provides ability to build portable objects dynamically
+    /// without having class definitions.
+    /// <para />
+    /// Note that type ID is required in order to build portable object. Usually it is
+    /// enough to provide a simple type name and Ignite will generate the type ID
+    /// automatically.
+    /// </summary>
+    public interface IPortableBuilder
+    {
+        /// <summary>
+        /// Get object field value. If value is another portable object, then
+        /// builder for this object will be returned. If value is a container
+        /// for other objects (array, ICollection, IDictionary), then container
+        /// will be returned with primitive types in deserialized form and
+        /// portable objects as builders. Any change in builder or collection
+        /// returned through this method will be reflected in the resulting
+        /// portable object after build.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Field value.</returns>
+        T GetField<T>(string fieldName);
+
+        /// <summary>
+        /// Set object field value. Value can be of any type including other
+        /// <see cref="IPortableObject"/> and other builders.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <param name="val">Field value.</param>
+        /// <returns>Current builder instance.</returns>
+        IPortableBuilder SetField<T>(string fieldName, T val);
+
+        /// <summary>
+        /// Remove object field.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Current builder instance.</returns>
+        IPortableBuilder RemoveField(string fieldName);
+
+        /// <summary>
+        /// Set explicit hash code. If builder creating object from scratch,
+        /// then hash code initially set to 0. If builder is created from
+        /// exising portable object, then hash code of that object is used
+        /// as initial value.
+        /// </summary>
+        /// <param name="hashCode">Hash code.</param>
+        /// <returns>Current builder instance.</returns>
+        [SuppressMessage("Microsoft.Naming", "CA1719:ParameterNamesShouldNotMatchMemberNames", MessageId = "0#")]
+        IPortableBuilder HashCode(int hashCode);
+
+        /// <summary>
+        /// Build the object.
+        /// </summary>
+        /// <returns>Resulting portable object.</returns>
+        IPortableObject Build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableIdMapper.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableIdMapper.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableIdMapper.cs
new file mode 100644
index 0000000..08c05b5
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableIdMapper.cs
@@ -0,0 +1,40 @@
+/*
+ * 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.Portable
+{
+    /// <summary>
+    /// Maps class name and class field names to integer identifiers.
+    /// </summary>
+    public interface IPortableIdMapper
+    {
+        /// <summary>
+        /// Gets type ID for the given type.
+        /// </summary>
+        /// <param name="typeName">Full type name.</param>
+        /// <returns>ID of the class or 0 in case hash code is to be used.</returns>
+        int TypeId(string typeName);
+
+        /// <summary>
+        /// Gets field ID for the given field of the given class.
+        /// </summary>
+        /// <param name="typeId">Type ID.</param>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>ID of the field or null in case hash code is to be used.</returns>
+        int FieldId(int typeId, string fieldName);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableMarshalAware.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableMarshalAware.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableMarshalAware.cs
new file mode 100644
index 0000000..2795db4
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableMarshalAware.cs
@@ -0,0 +1,39 @@
+/*
+ * 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.Portable
+{
+    /// <summary>
+    /// Interface to implement custom portable serialization logic.
+    /// </summary>
+    public interface IPortableMarshalAware 
+    {
+        /// <summary>
+        /// Writes this object to the given writer.
+        /// </summary> 
+        /// <param name="writer">Writer.</param>
+        /// <exception cref="System.IO.IOException">If write failed.</exception>
+        void WritePortable(IPortableWriter writer);
+
+        /// <summary>
+        /// Reads this object from the given reader.
+        /// </summary> 
+        /// <param name="reader">Reader.</param>
+        /// <exception cref="System.IO.IOException">If read failed.</exception>
+        void ReadPortable(IPortableReader reader);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableMetadata.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableMetadata.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableMetadata.cs
new file mode 100644
index 0000000..fa0e4f6
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableMetadata.cs
@@ -0,0 +1,61 @@
+/*
+ * 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.Portable
+{
+    using System.Collections.Generic;
+
+    /// <summary>
+    /// Portable type metadata.
+    /// </summary>
+    public interface IPortableMetadata
+    {
+        /// <summary>
+        /// Gets type name.
+        /// </summary>
+        /// <returns>Type name.</returns>
+        string TypeName
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Gets field names for that type.
+        /// </summary>
+        /// <returns>Field names.</returns>
+        ICollection<string> Fields
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Gets field type for the given field name.
+        /// </summary>
+        /// <param name="fieldName">Field name.</param>
+        /// <returns>Field type.</returns>
+        string FieldTypeName(string fieldName);
+
+        /// <summary>
+        /// Gets optional affinity key field name.
+        /// </summary>
+        /// <returns>Affinity key field name or null in case it is not provided.</returns>
+        string AffinityKeyFieldName
+        {
+            get;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableNameMapper.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableNameMapper.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableNameMapper.cs
new file mode 100644
index 0000000..5da824f
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Portable/IPortableNameMapper.cs
@@ -0,0 +1,39 @@
+/*
+ * 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.Portable
+{
+    /// <summary>
+    /// Maps type and field names to different names.
+    /// </summary>
+    public interface IPortableNameMapper
+    {
+        /// <summary>
+        /// Gets the type name.
+        /// </summary>
+        /// <param name="name">The name.</param>
+        /// <returns>Type name.</returns>
+        string TypeName(string name);
+
+        /// <summary>
+        /// Gets the field name.
+        /// </summary>
+        /// <param name="name">The name.</param>
+        /// <returns>Field name.</returns>
+        string FieldName(string name);
+    }
+}