You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2016/02/03 13:28:09 UTC

[14/28] ignite git commit: IGNITE-1906: .NET: Implemented programmatic configuration.

IGNITE-1906: .NET: Implemented programmatic configuration.


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

Branch: refs/heads/ignite-2508
Commit: ee20f1d9fb94f792d0b62097499c7c3f976ff6f5
Parents: 28a5247
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Tue Feb 2 16:19:55 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Feb 2 16:19:55 2016 +0300

----------------------------------------------------------------------
 .gitignore                                      |   1 +
 .../internal/binary/GridBinaryMarshaller.java   |  23 +-
 .../processors/cache/GridCacheProcessor.java    |  43 +-
 .../processors/platform/PlatformIgnition.java   |   6 +-
 .../platform/PlatformNoopProcessor.java         |  15 +
 .../processors/platform/PlatformProcessor.java  |  25 +
 .../platform/PlatformProcessorImpl.java         |  34 +-
 .../platform/cache/PlatformCache.java           |  16 +-
 .../dotnet/PlatformDotNetCacheStore.java        |  39 +-
 .../PlatformDotNetConfigurationClosure.java     |  50 +-
 .../utils/PlatformConfigurationUtils.java       | 621 +++++++++++++++++++
 .../platform/utils/PlatformUtils.java           |  52 +-
 .../PlatformDotNetCacheStoreFactoryNative.java  |  58 ++
 .../cpp/common/include/ignite/common/exports.h  |   3 +
 .../cpp/common/include/ignite/common/java.h     |   9 +
 .../platforms/cpp/common/project/vs/module.def  |   5 +-
 modules/platforms/cpp/common/src/exports.cpp    |  12 +
 modules/platforms/cpp/common/src/java.cpp       |  44 ++
 .../Apache.Ignite.Core.Tests.csproj             |   3 +
 .../Binary/BinarySelfTest.cs                    |   9 +
 .../Cache/CacheAbstractTest.cs                  |   2 +-
 .../Cache/CacheAffinityTest.cs                  |   2 +-
 .../Cache/CacheConfigurationTest.cs             | 538 ++++++++++++++++
 .../Cache/CacheDynamicStartTest.cs              |   4 +-
 .../Cache/CacheTestAsyncWrapper.cs              |   7 +
 .../Query/CacheQueriesCodeConfigurationTest.cs  | 295 +++++++++
 .../Cache/Query/CacheQueriesTest.cs             |   2 +-
 .../Continuous/ContinuousQueryAbstractTest.cs   |   2 +-
 .../Cache/Store/CacheStoreSessionTest.cs        |   2 +-
 .../Cache/Store/CacheStoreTest.cs               |  23 +-
 .../Dataload/DataStreamerTest.cs                |   4 +-
 .../Apache.Ignite.Core.Tests/ExceptionsTest.cs  |   2 +-
 .../IgniteConfigurationTest.cs                  | 367 +++++++++++
 .../Apache.Ignite.Core.Tests/MarshallerTest.cs  |   4 +-
 .../SerializationTest.cs                        |   2 +-
 .../Apache.Ignite.Core.Tests/TestRunner.cs      |   5 +-
 .../Apache.Ignite.Core.csproj                   |  29 +-
 .../Binary/BinaryConfiguration.cs               |  27 +-
 .../Configuration/CacheAtomicWriteOrderMode.cs  |  43 ++
 .../Cache/Configuration/CacheAtomicityMode.cs   |  54 ++
 .../Cache/Configuration/CacheConfiguration.cs   | 601 ++++++++++++++++++
 .../Cache/Configuration/CacheMemoryMode.cs      |  60 ++
 .../Cache/Configuration/CacheMode.cs            |  52 ++
 .../Cache/Configuration/CacheRebalanceMode.cs   |  51 ++
 .../CacheWriteSynchronizationMode.cs            |  45 ++
 .../Cache/Configuration/QueryAlias.cs           |  59 ++
 .../Cache/Configuration/QueryEntity.cs          | 401 ++++++++++++
 .../Cache/Configuration/QueryField.cs           | 109 ++++
 .../Cache/Configuration/QueryIndex.cs           | 137 ++++
 .../Cache/Configuration/QueryIndexField.cs      |  66 ++
 .../Cache/Configuration/QueryIndexType.cs       |  40 ++
 .../Configuration/QuerySqlFieldAttribute.cs     |  60 ++
 .../Configuration/QueryTextFieldAttribute.cs    |  36 ++
 .../dotnet/Apache.Ignite.Core/Cache/ICache.cs   |   6 +
 .../Apache.Ignite.Core/Common/IFactory.cs       |  34 +
 .../Discovery/IDiscoverySpi.cs                  |  32 +
 .../Discovery/Tcp/ITcpDiscoveryIpFinder.cs      |  34 +
 .../Multicast/TcpDiscoveryMulticastIpFinder.cs  | 133 ++++
 .../Tcp/Static/TcpDiscoveryStaticIpFinder.cs    |  84 +++
 .../Discovery/Tcp/TcpDiscoveryIpFinderBase.cs   |  78 +++
 .../Discovery/Tcp/TcpDiscoverySpi.cs            | 144 +++++
 .../dotnet/Apache.Ignite.Core/IIgnite.cs        |  29 +-
 .../Apache.Ignite.Core/IgniteConfiguration.cs   | 347 ++++++++++-
 .../dotnet/Apache.Ignite.Core/Ignition.cs       |  29 +-
 .../Apache.Ignite.Core/Impl/Binary/Binary.cs    |   3 -
 .../Impl/Binary/BinaryReaderExtensions.cs       |  19 +
 .../Apache.Ignite.Core/Impl/Binary/JavaTypes.cs |  92 +++
 .../Impl/Binary/Marshaller.cs                   |   8 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |   9 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheOp.cs    |   3 +-
 .../Impl/Cache/Store/CacheStore.cs              |  20 +-
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |  48 ++
 .../Impl/IgniteConfigurationEx.cs               |  57 --
 .../Apache.Ignite.Core/Impl/IgniteManager.cs    |   5 +-
 .../Apache.Ignite.Core/Impl/IgniteProxy.cs      |  18 +-
 .../Apache.Ignite.Core/Impl/IgniteUtils.cs      |   2 +-
 .../Impl/Memory/PlatformRawMemory.cs            |   2 +-
 .../Impl/Transactions/TransactionsImpl.cs       |   2 +-
 .../Impl/Unmanaged/IgniteJniNativeMethods.cs    |  11 +-
 .../Impl/Unmanaged/UnmanagedUtils.cs            |  22 +-
 parent/pom.xml                                  |   1 +
 81 files changed, 5213 insertions(+), 258 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index ef12f3a..e4e061c 100644
--- a/.gitignore
+++ b/.gitignore
@@ -24,6 +24,7 @@ git-patch-prop-local.sh
 *.vcxproj.user
 *.sdf
 *.opensdf
+*.opendb
 **/cpp/**/vs/x64/
 **/cpp/**/vs/Win32/
 **/dotnet/**/obj/

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java
index da43558..8e138fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java
@@ -299,6 +299,15 @@ public class GridBinaryMarshaller {
     /**
      * Push binary context and return the old one.
      *
+     * @return Old binary context.
+     */
+    public BinaryContext pushContext() {
+        return pushContext(ctx);
+    }
+
+    /**
+     * Push binary context and return the old one.
+     *
      * @param ctx Binary context.
      * @return Old binary context.
      */
@@ -315,11 +324,23 @@ public class GridBinaryMarshaller {
      *
      * @param oldCtx Old binary context.
      */
-    private static void popContext(@Nullable BinaryContext oldCtx) {
+    public static void popContext(@Nullable BinaryContext oldCtx) {
         BINARY_CTX.set(oldCtx);
     }
 
     /**
+     * Creates a reader.
+     *
+     * @param stream Stream.
+     * @return Reader.
+     */
+    public BinaryReaderExImpl reader(BinaryInputStream stream) {
+        assert stream != null;
+
+        return new BinaryReaderExImpl(ctx, stream, null);
+    }
+
+    /**
      * Whether object must be deserialized anyway. I.e. it cannot be converted to BinaryObject.
      *
      * @param obj Object.

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 26d3d4f..5acad6c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -69,10 +69,13 @@ import org.apache.ignite.internal.IgniteComponentType;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteNodeAttributes;
 import org.apache.ignite.internal.IgniteTransactionsEx;
+import org.apache.ignite.internal.binary.BinaryContext;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.internal.binary.GridBinaryMarshaller;
 import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
 import org.apache.ignite.internal.processors.cache.datastructures.CacheDataStructuresManager;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCache;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
@@ -94,6 +97,7 @@ import org.apache.ignite.internal.processors.cache.store.CacheStoreManager;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTransactionsImpl;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionManager;
+import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
 import org.apache.ignite.internal.processors.plugin.CachePluginManager;
 import org.apache.ignite.internal.processors.query.GridQueryProcessor;
 import org.apache.ignite.internal.util.F0;
@@ -3409,23 +3413,38 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (val == null)
             return null;
 
-        if (val.getCacheStoreFactory() != null) {
+        IgniteCacheObjectProcessor objProc = ctx.cacheObjects();
+        BinaryContext oldCtx = null;
+
+        if (objProc instanceof CacheObjectBinaryProcessorImpl) {
+            GridBinaryMarshaller binMarsh = ((CacheObjectBinaryProcessorImpl)objProc).marshaller();
+
+            oldCtx = binMarsh == null ? null : binMarsh.pushContext();
+        }
+
+        try {
+            if (val.getCacheStoreFactory() != null) {
+                try {
+                    marshaller.unmarshal(marshaller.marshal(val.getCacheStoreFactory()),
+                        val.getCacheStoreFactory().getClass().getClassLoader());
+                }
+                catch (IgniteCheckedException e) {
+                    throw new IgniteCheckedException("Failed to validate cache configuration. " +
+                        "Cache store factory is not serializable. Cache name: " + U.maskName(val.getName()), e);
+                }
+            }
+
             try {
-                marshaller.unmarshal(marshaller.marshal(val.getCacheStoreFactory()),
-                    val.getCacheStoreFactory().getClass().getClassLoader());
+                return marshaller.unmarshal(marshaller.marshal(val), U.resolveClassLoader(ctx.config().getClassLoader()));
             }
             catch (IgniteCheckedException e) {
-                throw new IgniteCheckedException("Failed to validate cache configuration. " +
-                    "Cache store factory is not serializable. Cache name: " + U.maskName(val.getName()), e);
+                throw new IgniteCheckedException("Failed to validate cache configuration " +
+                    "(make sure all objects in cache configuration are serializable): " + U.maskName(val.getName()), e);
             }
         }
-
-        try {
-            return marshaller.unmarshal(marshaller.marshal(val), U.resolveClassLoader(ctx.config().getClassLoader()));
-        }
-        catch (IgniteCheckedException e) {
-            throw new IgniteCheckedException("Failed to validate cache configuration " +
-                "(make sure all objects in cache configuration are serializable): " + U.maskName(val.getName()), e);
+        finally {
+            if (objProc instanceof CacheObjectBinaryProcessorImpl)
+                GridBinaryMarshaller.popContext(oldCtx);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformIgnition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformIgnition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformIgnition.java
index e642b2d..d25acfc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformIgnition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformIgnition.java
@@ -142,9 +142,11 @@ public class PlatformIgnition {
      * @return Configuration.
      */
     private static IgniteConfiguration configuration(@Nullable String springCfgPath) {
+        if (springCfgPath == null)
+            return new IgniteConfiguration();
+
         try {
-            URL url = springCfgPath == null ? U.resolveIgniteUrl(IgnitionEx.DFLT_CFG) :
-                U.resolveSpringUrl(springCfgPath);
+            URL url = U.resolveSpringUrl(springCfgPath);
 
             IgniteBiTuple<IgniteConfiguration, GridSpringResourceContext> t = IgnitionEx.loadConfiguration(url);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
index fb28008..b25e32e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
@@ -74,6 +74,16 @@ public class PlatformNoopProcessor extends GridProcessorAdapter implements Platf
     }
 
     /** {@inheritDoc} */
+    @Override public PlatformTarget createCacheFromConfig(long memPtr) throws IgniteCheckedException {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformTarget getOrCreateCacheFromConfig(long memPtr) throws IgniteCheckedException {
+        return null;
+    }
+
+    /** {@inheritDoc} */
     @Override public void destroyCache(@Nullable String name) throws IgniteCheckedException {
         // No-op.
     }
@@ -133,4 +143,9 @@ public class PlatformNoopProcessor extends GridProcessorAdapter implements Platf
     @Override public PlatformTarget atomicLong(String name, long initVal, boolean create) throws IgniteException {
         return null;
     }
+
+    /** {@inheritDoc} */
+    @Override public void getIgniteConfiguration(long memPtr) {
+        // No-op.
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
index 8e684e3..b59d93d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
@@ -90,6 +90,24 @@ public interface PlatformProcessor extends GridProcessor {
     public PlatformTarget getOrCreateCache(@Nullable String name) throws IgniteCheckedException;
 
     /**
+     * Create cache.
+     *
+     * @param memPtr Stream with cache config.
+     * @return Cache.
+     * @throws IgniteCheckedException If failed.
+     */
+    public PlatformTarget createCacheFromConfig(long memPtr) throws IgniteCheckedException;
+
+    /**
+     * Get or create cache.
+     *
+     * @param memPtr Stream with cache config.
+     * @return Cache.
+     * @throws IgniteCheckedException If failed.
+     */
+    public PlatformTarget getOrCreateCacheFromConfig(long memPtr) throws IgniteCheckedException;
+
+    /**
      * Destroy dynamically created cache.
      *
      * @param name Cache name.
@@ -188,4 +206,11 @@ public interface PlatformProcessor extends GridProcessor {
      * @throws IgniteException
      */
     public PlatformTarget atomicLong(String name, long initVal, boolean create) throws IgniteException;
+
+    /**
+     * Gets the configuration of the current Ignite instance.
+     *
+     * @param memPtr Stream to write data to.
+     */
+    public void getIgniteConfiguration(long memPtr);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
index dc6e0df..4ed8c25 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
@@ -22,11 +22,12 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.PlatformConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteComputeImpl;
+import org.apache.ignite.internal.binary.*;
 import org.apache.ignite.internal.cluster.ClusterGroupAdapter;
-import org.apache.ignite.internal.binary.BinaryRawWriterEx;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl;
@@ -45,6 +46,7 @@ import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStrea
 import org.apache.ignite.internal.processors.platform.messaging.PlatformMessaging;
 import org.apache.ignite.internal.processors.platform.services.PlatformServices;
 import org.apache.ignite.internal.processors.platform.transactions.PlatformTransactions;
+import org.apache.ignite.internal.processors.platform.utils.PlatformConfigurationUtils;
 import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -244,6 +246,26 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
     }
 
     /** {@inheritDoc} */
+    @Override public PlatformTarget createCacheFromConfig(long memPtr) throws IgniteCheckedException {
+        BinaryRawReaderEx reader = platformCtx.reader(platformCtx.memory().get(memPtr));
+        CacheConfiguration cfg = PlatformConfigurationUtils.readCacheConfiguration(reader);
+
+        IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().createCache(cfg);
+
+        return new PlatformCache(platformCtx, cache.keepBinary(), false);
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformTarget getOrCreateCacheFromConfig(long memPtr) throws IgniteCheckedException {
+        BinaryRawReaderEx reader = platformCtx.reader(platformCtx.memory().get(memPtr));
+        CacheConfiguration cfg = PlatformConfigurationUtils.readCacheConfiguration(reader);
+
+        IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().getOrCreateCache(cfg);
+
+        return new PlatformCache(platformCtx, cache.keepBinary(), false);
+    }
+
+    /** {@inheritDoc} */
     @Override public void destroyCache(@Nullable String name) throws IgniteCheckedException {
         ctx.grid().destroyCache(name);
     }
@@ -338,6 +360,16 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
         return new PlatformAtomicLong(platformCtx, atomicLong);
     }
 
+    /** {@inheritDoc} */
+    @Override public void getIgniteConfiguration(long memPtr) {
+        PlatformOutputStream stream = platformCtx.memory().get(memPtr).output();
+        BinaryRawWriterEx writer = platformCtx.writer(stream);
+
+        PlatformConfigurationUtils.writeIgniteConfiguration(writer, ignite().configuration());
+
+        stream.synchronize();
+    }
+
     /**
      * Internal store initialization routine.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
index 8e7c51d..37fd335 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
@@ -28,6 +28,7 @@ import org.apache.ignite.cache.query.ScanQuery;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.SqlQuery;
 import org.apache.ignite.cache.query.TextQuery;
+import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.binary.BinaryRawWriterEx;
@@ -41,6 +42,7 @@ import org.apache.ignite.internal.processors.platform.PlatformNativeException;
 import org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQuery;
 import org.apache.ignite.internal.processors.platform.cache.query.PlatformFieldsQueryCursor;
 import org.apache.ignite.internal.processors.platform.cache.query.PlatformQueryCursor;
+import org.apache.ignite.internal.processors.platform.utils.PlatformConfigurationUtils;
 import org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils;
 import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
 import org.apache.ignite.internal.util.GridConcurrentFactory;
@@ -178,6 +180,9 @@ public class PlatformCache extends PlatformAbstractTarget {
     /** */
     public static final int OP_REPLACE_3 = 38;
 
+    /** */
+    public static final int OP_GET_CONFIG = 39;
+
     /** Underlying JCache. */
     private final IgniteCacheProxy cache;
 
@@ -516,6 +521,14 @@ public class PlatformCache extends PlatformAbstractTarget {
 
                 break;
 
+            case OP_GET_CONFIG:
+                CacheConfiguration ccfg = ((IgniteCache<Object, Object>)cache).
+                        getConfiguration(CacheConfiguration.class);
+
+                PlatformConfigurationUtils.writeCacheConfiguration(writer, ccfg);
+
+                break;
+
             default:
                 super.processOutStream(type, writer);
         }
@@ -705,8 +718,7 @@ public class PlatformCache extends PlatformAbstractTarget {
     }
 
     /**
-     * Clears the contents of the cache, without notifying listeners or
-     * {@ignitelink javax.cache.integration.CacheWriter}s.
+     * Clears the contents of the cache, without notifying listeners or CacheWriters.
      *
      * @throws IllegalStateException if the cache is closed.
      * @throws javax.cache.CacheException if there is a problem during the clear

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java
index 47b1110..45d9208 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java
@@ -29,11 +29,8 @@ import org.apache.ignite.internal.processors.platform.cache.store.PlatformCacheS
 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.lang.GridMapEntry;
 import org.apache.ignite.internal.util.lang.GridTuple;
 import org.apache.ignite.internal.util.lang.IgniteInClosureX;
-import org.apache.ignite.internal.util.typedef.C1;
-import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiInClosure;
@@ -43,13 +40,9 @@ import org.jetbrains.annotations.Nullable;
 import javax.cache.Cache;
 import javax.cache.integration.CacheLoaderException;
 import javax.cache.integration.CacheWriterException;
-import java.util.AbstractMap;
-import java.util.AbstractSet;
 import java.util.Collection;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.Map;
-import java.util.Set;
 
 /**
  * Wrapper for .NET cache store implementations.
@@ -102,6 +95,9 @@ public class PlatformDotNetCacheStore<K, V> implements CacheStore<K, V>, Platfor
     /** Properties. */
     private Map<String, ?> props;
 
+    /** Native factory. */
+    private final Object nativeFactory;
+
     /** Interop processor. */
     protected PlatformContext platformCtx;
 
@@ -109,6 +105,22 @@ public class PlatformDotNetCacheStore<K, V> implements CacheStore<K, V>, Platfor
     protected long ptr;
 
     /**
+     * Default ctor.
+     */
+    public PlatformDotNetCacheStore() {
+        nativeFactory = null;
+    }
+
+    /**
+     * Native factory ctor.
+     */
+    public PlatformDotNetCacheStore(Object nativeFactory) {
+        assert nativeFactory != null;
+
+        this.nativeFactory = nativeFactory;
+    }
+
+    /**
      * Gets .NET class name.
      *
      * @return .NET class name.
@@ -175,7 +187,7 @@ public class PlatformDotNetCacheStore<K, V> implements CacheStore<K, V>, Platfor
                     writer.writeByte(OP_LOAD_ALL);
                     writer.writeLong(session());
                     writer.writeString(ses.cacheName());
-                    writer.writeCollection((Collection) keys);
+                    writer.writeCollection((Collection)keys);
                 }
             }, new LoadAllCallback<>(platformCtx, loaded));
 
@@ -305,7 +317,8 @@ public class PlatformDotNetCacheStore<K, V> implements CacheStore<K, V>, Platfor
      * @throws org.apache.ignite.IgniteCheckedException
      */
     public void initialize(GridKernalContext ctx, boolean convertBinary) throws IgniteCheckedException {
-        A.notNull(typName, "typName");
+        A.ensure(typName != null || nativeFactory != null,
+                "Either typName or nativeFactory must be set in PlatformDotNetCacheStore");
 
         platformCtx = PlatformUtils.platformContext(ctx.grid());
 
@@ -329,9 +342,13 @@ public class PlatformDotNetCacheStore<K, V> implements CacheStore<K, V>, Platfor
      * @param convertBinary Convert binary flag.
      */
     protected void write(BinaryRawWriterEx writer, boolean convertBinary) {
-        writer.writeString(typName);
         writer.writeBoolean(convertBinary);
-        writer.writeMap(props);
+        writer.writeObjectDetached(nativeFactory);
+
+        if (nativeFactory == null) {
+            writer.writeString(typName);
+            writer.writeMap(props);
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
index 6b9b441..8728d77 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
@@ -19,31 +19,28 @@ package org.apache.ignite.internal.processors.platform.dotnet;
 
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.binary.BinaryIdMapper;
 import org.apache.ignite.binary.BinaryBasicIdMapper;
-import org.apache.ignite.binary.BinaryNameMapper;
 import org.apache.ignite.binary.BinaryBasicNameMapper;
+import org.apache.ignite.binary.BinaryIdMapper;
+import org.apache.ignite.binary.BinaryNameMapper;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.PlatformConfiguration;
-import org.apache.ignite.internal.MarshallerContextImpl;
-import org.apache.ignite.internal.binary.BinaryNoopMetadataHandler;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.binary.BinaryRawWriterEx;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
 import org.apache.ignite.internal.binary.GridBinaryMarshaller;
-import org.apache.ignite.internal.binary.BinaryContext;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractConfigurationClosure;
 import org.apache.ignite.internal.processors.platform.lifecycle.PlatformLifecycleBean;
-import org.apache.ignite.internal.processors.platform.memory.PlatformInputStream;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemoryManagerImpl;
 import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
+import org.apache.ignite.internal.processors.platform.utils.PlatformConfigurationUtils;
 import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lifecycle.LifecycleBean;
-import org.apache.ignite.logger.NullLogger;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.platform.dotnet.PlatformDotNetConfiguration;
-import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.platform.dotnet.PlatformDotNetLifecycleBean;
 
 import java.util.ArrayList;
@@ -177,15 +174,16 @@ public class PlatformDotNetConfigurationClosure extends PlatformAbstractConfigur
      */
     @SuppressWarnings("ConstantConditions")
     private void prepare(IgniteConfiguration igniteCfg, PlatformDotNetConfigurationEx interopCfg) {
-        this.cfg = igniteCfg;
+        cfg = igniteCfg;
 
         try (PlatformMemory outMem = memMgr.allocate()) {
             try (PlatformMemory inMem = memMgr.allocate()) {
                 PlatformOutputStream out = outMem.output();
 
-                BinaryRawWriterEx writer = marshaller().writer(out);
+                GridBinaryMarshaller marshaller = PlatformUtils.marshaller();
+                BinaryRawWriterEx writer = marshaller.writer(out);
 
-                PlatformUtils.writeDotNetConfiguration(writer, interopCfg.unwrap());
+                PlatformConfigurationUtils.writeDotNetConfiguration(writer, interopCfg.unwrap());
 
                 List<PlatformDotNetLifecycleBean> beans = beans(igniteCfg);
 
@@ -201,7 +199,7 @@ public class PlatformDotNetConfigurationClosure extends PlatformAbstractConfigur
                 gate.extensionCallbackInLongLongOutLong(
                     PlatformUtils.OP_PREPARE_DOT_NET, outMem.pointer(), inMem.pointer());
 
-                processPrepareResult(inMem.input());
+                processPrepareResult(marshaller.reader(inMem.input()));
             }
         }
     }
@@ -211,9 +209,11 @@ public class PlatformDotNetConfigurationClosure extends PlatformAbstractConfigur
      *
      * @param in Input stream.
      */
-    private void processPrepareResult(PlatformInputStream in) {
+    private void processPrepareResult(BinaryReaderExImpl in) {
         assert cfg != null;
 
+        PlatformConfigurationUtils.readIgniteConfiguration(in, cfg);
+
         List<PlatformDotNetLifecycleBean> beans = beans(cfg);
         List<PlatformLifecycleBean> newBeans = new ArrayList<>();
 
@@ -265,28 +265,4 @@ public class PlatformDotNetConfigurationClosure extends PlatformAbstractConfigur
 
         return res;
     }
-
-    /**
-     * Create binary marshaller.
-     *
-     * @return Marshaller.
-     */
-    @SuppressWarnings("deprecation")
-    private static GridBinaryMarshaller marshaller() {
-        try {
-            BinaryContext ctx =
-                new BinaryContext(BinaryNoopMetadataHandler.instance(), new IgniteConfiguration(), new NullLogger());
-
-            BinaryMarshaller marsh = new BinaryMarshaller();
-
-            marsh.setContext(new MarshallerContextImpl(null));
-
-            ctx.configure(marsh, new IgniteConfiguration());
-
-            return new GridBinaryMarshaller(ctx);
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
new file mode 100644
index 0000000..32ab812
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
@@ -0,0 +1,621 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.platform.utils;
+
+import org.apache.ignite.binary.BinaryRawReader;
+import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMemoryMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheRebalanceMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.cache.QueryIndexType;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.binary.*;
+import org.apache.ignite.platform.dotnet.PlatformDotNetBinaryConfiguration;
+import org.apache.ignite.platform.dotnet.PlatformDotNetBinaryTypeConfiguration;
+import org.apache.ignite.platform.dotnet.PlatformDotNetCacheStoreFactoryNative;
+import org.apache.ignite.platform.dotnet.PlatformDotNetConfiguration;
+import org.apache.ignite.spi.discovery.DiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+
+import java.lang.management.ManagementFactory;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Configuration utils.
+ */
+@SuppressWarnings("unchecked") public class PlatformConfigurationUtils {
+    /**
+     * Write .Net configuration to the stream.
+     *
+     * @param writer Writer.
+     * @param cfg Configuration.
+     */
+    public static void writeDotNetConfiguration(BinaryRawWriterEx writer, PlatformDotNetConfiguration cfg) {
+        // 1. Write assemblies.
+        PlatformUtils.writeNullableCollection(writer, cfg.getAssemblies());
+
+        PlatformDotNetBinaryConfiguration binaryCfg = cfg.getBinaryConfiguration();
+
+        if (binaryCfg != null) {
+            writer.writeBoolean(true);
+
+            PlatformUtils.writeNullableCollection(writer, binaryCfg.getTypesConfiguration(),
+                new PlatformWriterClosure<PlatformDotNetBinaryTypeConfiguration>() {
+                    @Override public void write(BinaryRawWriterEx writer, PlatformDotNetBinaryTypeConfiguration typ) {
+                        writer.writeString(typ.getTypeName());
+                        writer.writeString(typ.getNameMapper());
+                        writer.writeString(typ.getIdMapper());
+                        writer.writeString(typ.getSerializer());
+                        writer.writeString(typ.getAffinityKeyFieldName());
+                        writer.writeObject(typ.getKeepDeserialized());
+                        writer.writeBoolean(typ.isEnum());
+                    }
+                });
+
+            PlatformUtils.writeNullableCollection(writer, binaryCfg.getTypes());
+            writer.writeString(binaryCfg.getDefaultNameMapper());
+            writer.writeString(binaryCfg.getDefaultIdMapper());
+            writer.writeString(binaryCfg.getDefaultSerializer());
+            writer.writeBoolean(binaryCfg.isDefaultKeepDeserialized());
+        }
+        else
+            writer.writeBoolean(false);
+    }
+
+    /**
+     * Reads cache configuration from a stream.
+     *
+     * @param in Stream.
+     * @return Cache configuration.
+     */
+    public static CacheConfiguration readCacheConfiguration(BinaryRawReaderEx in) {
+        assert in != null;
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setAtomicityMode(CacheAtomicityMode.fromOrdinal(in.readInt()));
+        ccfg.setAtomicWriteOrderMode(CacheAtomicWriteOrderMode.fromOrdinal((byte)in.readInt()));
+        ccfg.setBackups(in.readInt());
+        ccfg.setCacheMode(CacheMode.fromOrdinal(in.readInt()));
+        ccfg.setCopyOnRead(in.readBoolean());
+        ccfg.setEagerTtl(in.readBoolean());
+        ccfg.setSwapEnabled(in.readBoolean());
+        ccfg.setEvictSynchronized(in.readBoolean());
+        ccfg.setEvictSynchronizedConcurrencyLevel(in.readInt());
+        ccfg.setEvictSynchronizedKeyBufferSize(in.readInt());
+        ccfg.setEvictSynchronizedTimeout(in.readLong());
+        ccfg.setInvalidate(in.readBoolean());
+        ccfg.setStoreKeepBinary(in.readBoolean());
+        ccfg.setLoadPreviousValue(in.readBoolean());
+        ccfg.setDefaultLockTimeout(in.readLong());
+        ccfg.setLongQueryWarningTimeout(in.readLong());
+        ccfg.setMaxConcurrentAsyncOperations(in.readInt());
+        ccfg.setEvictMaxOverflowRatio(in.readFloat());
+        ccfg.setMemoryMode(CacheMemoryMode.values()[in.readInt()]);
+        ccfg.setName(in.readString());
+        ccfg.setOffHeapMaxMemory(in.readLong());
+        ccfg.setReadFromBackup(in.readBoolean());
+        ccfg.setRebalanceBatchSize(in.readInt());
+        ccfg.setRebalanceDelay(in.readLong());
+        ccfg.setRebalanceMode(CacheRebalanceMode.fromOrdinal(in.readInt()));
+        ccfg.setRebalanceThrottle(in.readLong());
+        ccfg.setRebalanceTimeout(in.readLong());
+        ccfg.setSqlEscapeAll(in.readBoolean());
+        ccfg.setSqlOnheapRowCacheSize(in.readInt());
+        ccfg.setStartSize(in.readInt());
+        ccfg.setWriteBehindBatchSize(in.readInt());
+        ccfg.setWriteBehindEnabled(in.readBoolean());
+        ccfg.setWriteBehindFlushFrequency(in.readLong());
+        ccfg.setWriteBehindFlushSize(in.readInt());
+        ccfg.setWriteBehindFlushThreadCount(in.readInt());
+        ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.fromOrdinal(in.readInt()));
+
+        Object storeFactory = in.readObjectDetached();
+
+        if (storeFactory != null)
+            ccfg.setCacheStoreFactory(new PlatformDotNetCacheStoreFactoryNative(storeFactory));
+
+        int qryEntCnt = in.readInt();
+
+        if (qryEntCnt > 0) {
+            Collection<QueryEntity> entities = new ArrayList<>(qryEntCnt);
+
+            for (int i = 0; i < qryEntCnt; i++)
+                entities.add(readQueryEntity(in));
+
+            ccfg.setQueryEntities(entities);
+        }
+
+        return ccfg;
+    }
+
+    /**
+     * Reads the query entity.
+     *
+     * @param in Stream.
+     * @return QueryEntity.
+     */
+    public static QueryEntity readQueryEntity(BinaryRawReader in) {
+        QueryEntity res = new QueryEntity();
+
+        res.setKeyType(in.readString());
+        res.setValueType(in.readString());
+
+        // Fields
+        int cnt = in.readInt();
+
+        if (cnt > 0) {
+            LinkedHashMap<String, String> fields = new LinkedHashMap<>(cnt);
+
+            for (int i = 0; i < cnt; i++)
+                fields.put(in.readString(), in.readString());
+
+            res.setFields(fields);
+        }
+
+        // Aliases
+        cnt = in.readInt();
+
+        if (cnt > 0) {
+            Map<String, String> aliases = new HashMap<>(cnt);
+
+            for (int i = 0; i < cnt; i++)
+                aliases.put(in.readString(), in.readString());
+
+            res.setAliases(aliases);
+        }
+
+        // Indexes
+        cnt = in.readInt();
+
+        if (cnt > 0) {
+            Collection<QueryIndex> indexes = new ArrayList<>(cnt);
+
+            for (int i = 0; i < cnt; i++)
+                indexes.add(readQueryIndex(in));
+
+            res.setIndexes(indexes);
+        }
+
+        return res;
+    }
+
+    /**
+     * Reads the query index.
+     *
+     * @param in Reader.
+     * @return Query index.
+     */
+    public static QueryIndex readQueryIndex(BinaryRawReader in) {
+        QueryIndex res = new QueryIndex();
+
+        res.setName(in.readString());
+        res.setIndexType(QueryIndexType.values()[in.readByte()]);
+
+        int cnt = in.readInt();
+
+        if (cnt > 0) {
+            LinkedHashMap<String, Boolean> fields = new LinkedHashMap<>(cnt);
+
+            for (int i = 0; i < cnt; i++)
+                fields.put(in.readString(), !in.readBoolean());
+
+            res.setFields(fields);
+        }
+
+        return res;
+    }
+
+    /**
+     * Reads Ignite configuration.
+     * @param in Reader.
+     * @param cfg Configuration.
+     */
+    public static void readIgniteConfiguration(BinaryRawReaderEx in, IgniteConfiguration cfg) {
+        if (!in.readBoolean())
+            return;  // there is no config
+
+        cfg.setClientMode(in.readBoolean());
+        cfg.setIncludeEventTypes(in.readIntArray());
+        cfg.setMetricsExpireTime(in.readLong());
+        cfg.setMetricsHistorySize(in.readInt());
+        cfg.setMetricsLogFrequency(in.readLong());
+        cfg.setMetricsUpdateFrequency(in.readLong());
+        cfg.setNetworkSendRetryCount(in.readInt());
+        cfg.setNetworkSendRetryDelay(in.readLong());
+        cfg.setNetworkTimeout(in.readLong());
+        cfg.setWorkDirectory(in.readString());
+        cfg.setLocalHost(in.readString());
+
+        readCacheConfigurations(in, cfg);
+        readDiscoveryConfiguration(in, cfg);
+    }
+
+    /**
+     * Reads cache configurations from a stream and updates provided IgniteConfiguration.
+     *
+     * @param cfg IgniteConfiguration to update.
+     * @param in Reader.
+     */
+    public static void readCacheConfigurations(BinaryRawReaderEx in, IgniteConfiguration cfg) {
+        int len = in.readInt();
+
+        if (len == 0)
+            return;
+
+        List<CacheConfiguration> caches = new ArrayList<>();
+
+        for (int i = 0; i < len; i++)
+            caches.add(readCacheConfiguration(in));
+
+        CacheConfiguration[] oldCaches = cfg.getCacheConfiguration();
+        CacheConfiguration[] caches0 = caches.toArray(new CacheConfiguration[caches.size()]);
+
+        if (oldCaches == null)
+            cfg.setCacheConfiguration(caches0);
+        else {
+            CacheConfiguration[] mergedCaches = new CacheConfiguration[oldCaches.length + caches.size()];
+
+            System.arraycopy(oldCaches, 0, mergedCaches, 0, oldCaches.length);
+            System.arraycopy(caches0, 0, mergedCaches, oldCaches.length, caches.size());
+
+            cfg.setCacheConfiguration(mergedCaches);
+        }
+    }
+
+    /**
+     * Reads discovery configuration from a stream and updates provided IgniteConfiguration.
+     *
+     * @param cfg IgniteConfiguration to update.
+     * @param in Reader.
+     */
+    public static void readDiscoveryConfiguration(BinaryRawReader in, IgniteConfiguration cfg) {
+        boolean hasConfig = in.readBoolean();
+
+        if (!hasConfig)
+            return;
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        boolean hasIpFinder = in.readBoolean();
+
+        if (hasIpFinder) {
+            byte ipFinderType = in.readByte();
+
+            int addrCount = in.readInt();
+
+            ArrayList<String> addrs = null;
+
+            if (addrCount > 0) {
+                addrs = new ArrayList<>(addrCount);
+
+                for (int i = 0; i < addrCount; i++)
+                    addrs.add(in.readString());
+            }
+
+            TcpDiscoveryVmIpFinder finder = null;
+            if (ipFinderType == 1) {
+                finder = new TcpDiscoveryVmIpFinder();
+            }
+            else if (ipFinderType == 2) {
+                TcpDiscoveryMulticastIpFinder finder0 = new TcpDiscoveryMulticastIpFinder();
+
+                finder0.setLocalAddress(in.readString());
+                finder0.setMulticastGroup(in.readString());
+                finder0.setMulticastPort(in.readInt());
+                finder0.setAddressRequestAttempts(in.readInt());
+                finder0.setResponseWaitTime(in.readInt());
+
+                boolean hasTtl = in.readBoolean();
+
+                if (hasTtl)
+                    finder0.setTimeToLive(in.readInt());
+
+                finder = finder0;
+            }
+            else {
+                assert false;
+            }
+
+            finder.setAddresses(addrs);
+
+            disco.setIpFinder(finder);
+        }
+
+        disco.setSocketTimeout(in.readLong());
+        disco.setAckTimeout(in.readLong());
+        disco.setMaxAckTimeout(in.readLong());
+        disco.setNetworkTimeout(in.readLong());
+        disco.setJoinTimeout(in.readLong());
+
+        cfg.setDiscoverySpi(disco);
+    }
+
+    /**
+     * Writes cache configuration.
+     *
+     * @param writer Writer.
+     * @param ccfg Configuration.
+     */
+    public static void writeCacheConfiguration(BinaryRawWriter writer, CacheConfiguration ccfg) {
+        assert writer != null;
+        assert ccfg != null;
+
+        writer.writeInt(ccfg.getAtomicityMode() == null ?
+            CacheConfiguration.DFLT_CACHE_ATOMICITY_MODE.ordinal() : ccfg.getAtomicityMode().ordinal());
+        writer.writeInt(ccfg.getAtomicWriteOrderMode() == null ? 0 : ccfg.getAtomicWriteOrderMode().ordinal());
+        writer.writeInt(ccfg.getBackups());
+        writer.writeInt(ccfg.getCacheMode() == null ?
+            CacheConfiguration.DFLT_CACHE_MODE.ordinal() : ccfg.getCacheMode().ordinal());
+        writer.writeBoolean(ccfg.isCopyOnRead());
+        writer.writeBoolean(ccfg.isEagerTtl());
+        writer.writeBoolean(ccfg.isSwapEnabled());
+        writer.writeBoolean(ccfg.isEvictSynchronized());
+        writer.writeInt(ccfg.getEvictSynchronizedConcurrencyLevel());
+        writer.writeInt(ccfg.getEvictSynchronizedKeyBufferSize());
+        writer.writeLong(ccfg.getEvictSynchronizedTimeout());
+        writer.writeBoolean(ccfg.isInvalidate());
+        writer.writeBoolean(ccfg.isStoreKeepBinary());
+        writer.writeBoolean(ccfg.isLoadPreviousValue());
+        writer.writeLong(ccfg.getDefaultLockTimeout());
+        writer.writeLong(ccfg.getLongQueryWarningTimeout());
+        writer.writeInt(ccfg.getMaxConcurrentAsyncOperations());
+        writer.writeFloat(ccfg.getEvictMaxOverflowRatio());
+        writer.writeInt(ccfg.getMemoryMode() == null ?
+            CacheConfiguration.DFLT_MEMORY_MODE.ordinal() : ccfg.getMemoryMode().ordinal());
+        writer.writeString(ccfg.getName());
+        writer.writeLong(ccfg.getOffHeapMaxMemory());
+        writer.writeBoolean(ccfg.isReadFromBackup());
+        writer.writeInt(ccfg.getRebalanceBatchSize());
+        writer.writeLong(ccfg.getRebalanceDelay());
+        writer.writeInt(ccfg.getRebalanceMode() == null ?
+            CacheConfiguration.DFLT_REBALANCE_MODE.ordinal() : ccfg.getRebalanceMode().ordinal());
+        writer.writeLong(ccfg.getRebalanceThrottle());
+        writer.writeLong(ccfg.getRebalanceTimeout());
+        writer.writeBoolean(ccfg.isSqlEscapeAll());
+        writer.writeInt(ccfg.getSqlOnheapRowCacheSize());
+        writer.writeInt(ccfg.getStartSize());
+        writer.writeInt(ccfg.getWriteBehindBatchSize());
+        writer.writeBoolean(ccfg.isWriteBehindEnabled());
+        writer.writeLong(ccfg.getWriteBehindFlushFrequency());
+        writer.writeInt(ccfg.getWriteBehindFlushSize());
+        writer.writeInt(ccfg.getWriteBehindFlushThreadCount());
+        writer.writeInt(ccfg.getWriteSynchronizationMode() == null ? 0 : ccfg.getWriteSynchronizationMode().ordinal());
+
+        if (ccfg.getCacheStoreFactory() instanceof PlatformDotNetCacheStoreFactoryNative)
+            writer.writeObject(((PlatformDotNetCacheStoreFactoryNative)ccfg.getCacheStoreFactory()).getNativeFactory());
+        else
+            writer.writeObject(null);
+
+        Collection<QueryEntity> qryEntities = ccfg.getQueryEntities();
+
+        if (qryEntities != null)
+        {
+            writer.writeInt(qryEntities.size());
+
+            for (QueryEntity e : qryEntities)
+                writeQueryEntity(writer, e);
+        }
+        else
+            writer.writeInt(0);
+    }
+
+    /**
+     * Write query entity.
+     *
+     * @param writer Writer.
+     * @param queryEntity Query entity.
+     */
+    private static void writeQueryEntity(BinaryRawWriter writer, QueryEntity queryEntity) {
+        assert queryEntity != null;
+
+        writer.writeString(queryEntity.getKeyType());
+        writer.writeString(queryEntity.getValueType());
+
+        // Fields
+        LinkedHashMap<String, String> fields = queryEntity.getFields();
+
+        if (fields != null) {
+            writer.writeInt(fields.size());
+
+            for (Map.Entry<String, String> field : fields.entrySet()) {
+                writer.writeString(field.getKey());
+                writer.writeString(field.getValue());
+            }
+        }
+        else
+            writer.writeInt(0);
+
+        // Aliases
+        Map<String, String> aliases = queryEntity.getAliases();
+
+        if (aliases != null) {
+            writer.writeInt(aliases.size());
+
+            for (Map.Entry<String, String> alias : aliases.entrySet()) {
+                writer.writeString(alias.getKey());
+                writer.writeString(alias.getValue());
+            }
+        }
+        else
+            writer.writeInt(0);
+
+        // Indexes
+        Collection<QueryIndex> indexes = queryEntity.getIndexes();
+
+        if (indexes != null) {
+            writer.writeInt(indexes.size());
+
+            for (QueryIndex index : indexes)
+                writeQueryIndex(writer, index);
+        }
+        else
+            writer.writeInt(0);
+    }
+
+    /**
+     * Writer query index.
+     *
+     * @param writer Writer.
+     * @param index Index.
+     */
+    private static void writeQueryIndex(BinaryRawWriter writer, QueryIndex index) {
+        assert index != null;
+
+        writer.writeString(index.getName());
+        writer.writeByte((byte)index.getIndexType().ordinal());
+
+        LinkedHashMap<String, Boolean> fields = index.getFields();
+
+        if (fields != null) {
+            writer.writeInt(fields.size());
+
+            for (Map.Entry<String, Boolean> field : fields.entrySet()) {
+                writer.writeString(field.getKey());
+                writer.writeBoolean(!field.getValue());
+            }
+        }
+        else
+            writer.writeInt(0);
+    }
+
+    /**
+     * Writes Ignite configuration.
+     *
+     * @param w Writer.
+     * @param cfg Configuration.
+     */
+    public static void writeIgniteConfiguration(BinaryRawWriter w, IgniteConfiguration cfg) {
+        assert w != null;
+        assert cfg != null;
+
+        w.writeBoolean(cfg.isClientMode());
+        w.writeIntArray(cfg.getIncludeEventTypes());
+        w.writeLong(cfg.getMetricsExpireTime());
+        w.writeInt(cfg.getMetricsHistorySize());
+        w.writeLong(cfg.getMetricsLogFrequency());
+        w.writeLong(cfg.getMetricsUpdateFrequency());
+        w.writeInt(cfg.getNetworkSendRetryCount());
+        w.writeLong(cfg.getNetworkSendRetryDelay());
+        w.writeLong(cfg.getNetworkTimeout());
+        w.writeString(cfg.getWorkDirectory());
+        w.writeString(cfg.getLocalHost());
+
+        CacheConfiguration[] cacheCfg = cfg.getCacheConfiguration();
+
+        if (cacheCfg != null) {
+            w.writeInt(cacheCfg.length);
+
+            for (CacheConfiguration ccfg : cacheCfg)
+                writeCacheConfiguration(w, ccfg);
+        }
+        else
+            w.writeInt(0);
+
+        writeDiscoveryConfiguration(w, cfg.getDiscoverySpi());
+
+        w.writeString(cfg.getIgniteHome());
+
+        w.writeLong(ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getInit());
+        w.writeLong(ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getMax());
+    }
+
+    /**
+     * Writes discovery configuration.
+     *
+     * @param w Writer.
+     * @param spi Disco.
+     */
+    private static void writeDiscoveryConfiguration(BinaryRawWriter w, DiscoverySpi spi) {
+        assert w != null;
+        assert spi != null;
+
+        if (!(spi instanceof TcpDiscoverySpi)) {
+            w.writeBoolean(false);
+            return;
+        }
+
+        w.writeBoolean(true);
+
+        TcpDiscoverySpi tcp = (TcpDiscoverySpi)spi;
+
+        TcpDiscoveryIpFinder finder = tcp.getIpFinder();
+
+        if (finder instanceof TcpDiscoveryVmIpFinder) {
+            w.writeBoolean(true);
+
+            boolean isMulticast = finder instanceof TcpDiscoveryMulticastIpFinder;
+
+            w.writeByte((byte)(isMulticast ? 2 : 1));
+
+            Collection<InetSocketAddress> addrs = finder.getRegisteredAddresses();
+
+            w.writeInt(addrs.size());
+
+            for (InetSocketAddress a : addrs)
+                w.writeString(a.toString());
+
+            if (isMulticast) {
+                TcpDiscoveryMulticastIpFinder multiFinder = (TcpDiscoveryMulticastIpFinder) finder;
+
+                w.writeString(multiFinder.getLocalAddress());
+                w.writeString(multiFinder.getMulticastGroup());
+                w.writeInt(multiFinder.getMulticastPort());
+                w.writeInt(multiFinder.getAddressRequestAttempts());
+                w.writeInt(multiFinder.getResponseWaitTime());
+
+                Integer ttl = multiFinder.getTimeToLive();
+                w.writeBoolean(ttl != null);
+
+                if (ttl != null)
+                    w.writeInt(ttl);
+            }
+        }
+        else {
+            w.writeBoolean(false);
+        }
+
+        w.writeLong(tcp.getSocketTimeout());
+        w.writeLong(tcp.getAckTimeout());
+        w.writeLong(tcp.getMaxAckTimeout());
+        w.writeLong(tcp.getNetworkTimeout());
+        w.writeLong(tcp.getJoinTimeout());
+    }
+
+    /**
+     * Private constructor.
+     */
+    private PlatformConfigurationUtils() {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java
index 4a29637..d09b2c7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java
@@ -22,10 +22,16 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.CachePeekMode;
+import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.MarshallerContextImpl;
+import org.apache.ignite.internal.binary.BinaryContext;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.internal.binary.BinaryNoopMetadataHandler;
 import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.binary.BinaryRawWriterEx;
+import org.apache.ignite.internal.binary.GridBinaryMarshaller;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.PlatformExtendedException;
 import org.apache.ignite.internal.processors.platform.PlatformNativeException;
@@ -38,9 +44,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.platform.dotnet.PlatformDotNetConfiguration;
-import org.apache.ignite.platform.dotnet.PlatformDotNetBinaryConfiguration;
-import org.apache.ignite.platform.dotnet.PlatformDotNetBinaryTypeConfiguration;
+import org.apache.ignite.logger.NullLogger;
 import org.jetbrains.annotations.Nullable;
 
 import javax.cache.CacheException;
@@ -764,41 +768,27 @@ public class PlatformUtils {
     }
 
     /**
-     * Write .Net configuration to the stream.
+     * Create binary marshaller.
      *
-     * @param writer Writer.
-     * @param cfg Configuration.
+     * @return Marshaller.
      */
-    public static void writeDotNetConfiguration(BinaryRawWriterEx writer, PlatformDotNetConfiguration cfg) {
-        // 1. Write assemblies.
-        writeNullableCollection(writer, cfg.getAssemblies());
+    @SuppressWarnings("deprecation")
+    public static GridBinaryMarshaller marshaller() {
+        try {
+            BinaryContext ctx =
+                new BinaryContext(BinaryNoopMetadataHandler.instance(), new IgniteConfiguration(), new NullLogger());
 
-        PlatformDotNetBinaryConfiguration binaryCfg = cfg.getBinaryConfiguration();
+            BinaryMarshaller marsh = new BinaryMarshaller();
 
-        if (binaryCfg != null) {
-            writer.writeBoolean(true);
+            marsh.setContext(new MarshallerContextImpl(null));
 
-            writeNullableCollection(writer, binaryCfg.getTypesConfiguration(),
-                new PlatformWriterClosure<PlatformDotNetBinaryTypeConfiguration>() {
-                @Override public void write(BinaryRawWriterEx writer, PlatformDotNetBinaryTypeConfiguration typ) {
-                    writer.writeString(typ.getTypeName());
-                    writer.writeString(typ.getNameMapper());
-                    writer.writeString(typ.getIdMapper());
-                    writer.writeString(typ.getSerializer());
-                    writer.writeString(typ.getAffinityKeyFieldName());
-                    writer.writeObject(typ.getKeepDeserialized());
-                    writer.writeBoolean(typ.isEnum());
-                }
-            });
+            ctx.configure(marsh, new IgniteConfiguration());
 
-            writeNullableCollection(writer, binaryCfg.getTypes());
-            writer.writeString(binaryCfg.getDefaultNameMapper());
-            writer.writeString(binaryCfg.getDefaultIdMapper());
-            writer.writeString(binaryCfg.getDefaultSerializer());
-            writer.writeBoolean(binaryCfg.isDefaultKeepDeserialized());
+            return new GridBinaryMarshaller(ctx);
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
         }
-        else
-            writer.writeBoolean(false);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetCacheStoreFactoryNative.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetCacheStoreFactoryNative.java b/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetCacheStoreFactoryNative.java
new file mode 100644
index 0000000..ec01483
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetCacheStoreFactoryNative.java
@@ -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.
+ */
+
+package org.apache.ignite.platform.dotnet;
+
+import org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore;
+
+import javax.cache.configuration.Factory;
+
+/**
+ * Cache store factory that wraps native factory object.
+ */
+public class PlatformDotNetCacheStoreFactoryNative implements Factory<PlatformDotNetCacheStore> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private final Object nativeFactory;
+
+    /**
+     * Ctor.
+     *
+     * @param nativeFactory Native factory object.
+     */
+    public PlatformDotNetCacheStoreFactoryNative(Object nativeFactory) {
+        assert nativeFactory != null;
+
+        this.nativeFactory = nativeFactory;
+    }
+
+    /**
+     * Gets the wrapped factory object.
+     *
+     * @return Factory object.
+     */
+    public Object getNativeFactory() {
+        return nativeFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformDotNetCacheStore create() {
+        return new PlatformDotNetCacheStore(nativeFactory);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/cpp/common/include/ignite/common/exports.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/ignite/common/exports.h b/modules/platforms/cpp/common/include/ignite/common/exports.h
index 67583ed..66f918f 100644
--- a/modules/platforms/cpp/common/include/ignite/common/exports.h
+++ b/modules/platforms/cpp/common/include/ignite/common/exports.h
@@ -36,6 +36,8 @@ extern "C" {
     void* IGNITE_CALL IgniteProcessorCache(gcj::JniContext* ctx, void* obj, char* name);
     void* IGNITE_CALL IgniteProcessorCreateCache(gcj::JniContext* ctx, void* obj, char* name);
     void* IGNITE_CALL IgniteProcessorGetOrCreateCache(gcj::JniContext* ctx, void* obj, char* name);
+    void* IGNITE_CALL IgniteProcessorCreateCacheFromConfig(gcj::JniContext* ctx, void* obj, long memPtr);
+    void* IGNITE_CALL IgniteProcessorGetOrCreateCacheFromConfig(gcj::JniContext* ctx, void* obj, long memPtr);
     void IGNITE_CALL IgniteProcessorDestroyCache(gcj::JniContext* ctx, void* obj, char* name);
     void* IGNITE_CALL IgniteProcessorAffinity(gcj::JniContext* ctx, void* obj, char* name);
     void* IGNITE_CALL IgniteProcessorDataStreamer(gcj::JniContext* ctx, void* obj, char* name, bool keepPortable);
@@ -46,6 +48,7 @@ extern "C" {
     void* IGNITE_CALL IgniteProcessorServices(gcj::JniContext* ctx, void* obj, void* prj);
     void* IGNITE_CALL IgniteProcessorExtensions(gcj::JniContext* ctx, void* obj);
     void* IGNITE_CALL IgniteProcessorAtomicLong(gcj::JniContext* ctx, void* obj, char* name, long long initVal, bool create);
+    void IGNITE_CALL IgniteProcessorGetIgniteConfiguration(gcj::JniContext* ctx, void* obj, long memPtr);
     
     long long IGNITE_CALL IgniteTargetInStreamOutLong(gcj::JniContext* ctx, void* obj, int opType, long long memPtr);
     void IGNITE_CALL IgniteTargetInStreamOutStream(gcj::JniContext* ctx, void* obj, int opType, long long inMemPtr, long long outMemPtr);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/cpp/common/include/ignite/common/java.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/ignite/common/java.h b/modules/platforms/cpp/common/include/ignite/common/java.h
index 572f040..072a8ef 100644
--- a/modules/platforms/cpp/common/include/ignite/common/java.h
+++ b/modules/platforms/cpp/common/include/ignite/common/java.h
@@ -297,6 +297,8 @@ namespace ignite
                 jmethodID m_PlatformProcessor_cache;
                 jmethodID m_PlatformProcessor_createCache;
                 jmethodID m_PlatformProcessor_getOrCreateCache;
+                jmethodID m_PlatformProcessor_createCacheFromConfig;
+                jmethodID m_PlatformProcessor_getOrCreateCacheFromConfig;
                 jmethodID m_PlatformProcessor_destroyCache;
                 jmethodID m_PlatformProcessor_affinity;
                 jmethodID m_PlatformProcessor_dataStreamer;
@@ -308,6 +310,7 @@ namespace ignite
                 jmethodID m_PlatformProcessor_services;
                 jmethodID m_PlatformProcessor_extensions;
                 jmethodID m_PlatformProcessor_atomicLong;
+                jmethodID m_PlatformProcessor_getIgniteConfiguration;
 
                 jclass c_PlatformTarget;
                 jmethodID m_PlatformTarget_inStreamOutLong;
@@ -489,6 +492,10 @@ namespace ignite
                 jobject ProcessorCreateCache(jobject obj, const char* name, JniErrorInfo* errInfo);
                 jobject ProcessorGetOrCreateCache(jobject obj, const char* name);
                 jobject ProcessorGetOrCreateCache(jobject obj, const char* name, JniErrorInfo* errInfo);
+                jobject ProcessorCreateCacheFromConfig(jobject obj, long memPtr);
+                jobject ProcessorCreateCacheFromConfig(jobject obj, long memPtr, JniErrorInfo* errInfo);
+                jobject ProcessorGetOrCreateCacheFromConfig(jobject obj, long memPtr);
+                jobject ProcessorGetOrCreateCacheFromConfig(jobject obj, long memPtr, JniErrorInfo* errInfo);
                 void ProcessorDestroyCache(jobject obj, const char* name);
                 void ProcessorDestroyCache(jobject obj, const char* name, JniErrorInfo* errInfo);
                 jobject ProcessorAffinity(jobject obj, const char* name);
@@ -500,6 +507,7 @@ namespace ignite
                 jobject ProcessorServices(jobject obj, jobject prj);
                 jobject ProcessorExtensions(jobject obj);
                 jobject ProcessorAtomicLong(jobject obj, char* name, long long initVal, bool create);
+				void ProcessorGetIgniteConfiguration(jobject obj, long memPtr);
                 
                 long long TargetInStreamOutLong(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL);
                 void TargetInStreamOutStream(jobject obj, int opType, long long inMemPtr, long long outMemPtr, JniErrorInfo* errInfo = NULL);
@@ -618,6 +626,7 @@ namespace ignite
                 void ExceptionCheck(JNIEnv* env, JniErrorInfo* errInfo);
                 jobject LocalToGlobal(JNIEnv* env, jobject obj);
                 jobject ProcessorCache0(jobject proc, const char* name, jmethodID mthd, JniErrorInfo* errInfo);
+                jobject ProcessorCacheFromConfig0(jobject proc, long memPtr, jmethodID mthd, JniErrorInfo* errInfo);
             };
 
             JNIEXPORT jlong JNICALL JniCacheStoreCreate(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/cpp/common/project/vs/module.def
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/project/vs/module.def b/modules/platforms/cpp/common/project/vs/module.def
index c947128..81df027 100644
--- a/modules/platforms/cpp/common/project/vs/module.def
+++ b/modules/platforms/cpp/common/project/vs/module.def
@@ -113,4 +113,7 @@ IgniteListenableCancel @110
 IgniteListenableIsCancelled @111
 IgniteTargetListenFutureAndGet @112
 IgniteTargetListenFutureForOperationAndGet @113
-IgniteProcessorDestroyCache @114
\ No newline at end of file
+IgniteProcessorCreateCacheFromConfig @114
+IgniteProcessorGetOrCreateCacheFromConfig @115
+IgniteProcessorGetIgniteConfiguration @116
+IgniteProcessorDestroyCache @117
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/cpp/common/src/exports.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/src/exports.cpp b/modules/platforms/cpp/common/src/exports.cpp
index d4ffa7e..e9ec519 100644
--- a/modules/platforms/cpp/common/src/exports.cpp
+++ b/modules/platforms/cpp/common/src/exports.cpp
@@ -66,6 +66,14 @@ extern "C" {
         return ctx->ProcessorGetOrCreateCache(static_cast<jobject>(obj), name);
     }
 
+    void* IGNITE_CALL IgniteProcessorCreateCacheFromConfig(gcj::JniContext* ctx, void* obj, long memPtr) {
+        return ctx->ProcessorCreateCacheFromConfig(static_cast<jobject>(obj), memPtr);
+    }
+
+    void* IGNITE_CALL IgniteProcessorGetOrCreateCacheFromConfig(gcj::JniContext* ctx, void* obj, long memPtr) {
+        return ctx->ProcessorGetOrCreateCacheFromConfig(static_cast<jobject>(obj), memPtr);
+    }
+
     void IGNITE_CALL IgniteProcessorDestroyCache(gcj::JniContext* ctx, void* obj, char* name) {
         ctx->ProcessorDestroyCache(static_cast<jobject>(obj), name);
     }
@@ -106,6 +114,10 @@ extern "C" {
         return ctx->ProcessorAtomicLong(static_cast<jobject>(obj), name, initVal, create);
     }
 
+	void IGNITE_CALL IgniteProcessorGetIgniteConfiguration(gcj::JniContext* ctx, void* obj, long memPtr) {
+        return ctx->ProcessorGetIgniteConfiguration(static_cast<jobject>(obj), memPtr);
+    }
+
     long long IGNITE_CALL IgniteTargetInStreamOutLong(gcj::JniContext* ctx, void* obj, int opType, long long memPtr) {
         return ctx->TargetInStreamOutLong(static_cast<jobject>(obj), opType, memPtr);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/cpp/common/src/java.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/src/java.cpp b/modules/platforms/cpp/common/src/java.cpp
index 9e55742..e36c1e0 100644
--- a/modules/platforms/cpp/common/src/java.cpp
+++ b/modules/platforms/cpp/common/src/java.cpp
@@ -191,6 +191,8 @@ namespace ignite
             JniMethod M_PLATFORM_PROCESSOR_CACHE = JniMethod("cache", "(Ljava/lang/String;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
             JniMethod M_PLATFORM_PROCESSOR_CREATE_CACHE = JniMethod("createCache", "(Ljava/lang/String;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
             JniMethod M_PLATFORM_PROCESSOR_GET_OR_CREATE_CACHE = JniMethod("getOrCreateCache", "(Ljava/lang/String;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
+            JniMethod M_PLATFORM_PROCESSOR_CREATE_CACHE_FROM_CONFIG = JniMethod("createCacheFromConfig", "(J)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
+            JniMethod M_PLATFORM_PROCESSOR_GET_OR_CREATE_CACHE_FROM_CONFIG = JniMethod("getOrCreateCacheFromConfig", "(J)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
             JniMethod M_PLATFORM_PROCESSOR_DESTROY_CACHE = JniMethod("destroyCache", "(Ljava/lang/String;)V", false);
             JniMethod M_PLATFORM_PROCESSOR_AFFINITY = JniMethod("affinity", "(Ljava/lang/String;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
             JniMethod M_PLATFORM_PROCESSOR_DATA_STREAMER = JniMethod("dataStreamer", "(Ljava/lang/String;Z)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
@@ -201,6 +203,7 @@ namespace ignite
             JniMethod M_PLATFORM_PROCESSOR_SERVICES = JniMethod("services", "(Lorg/apache/ignite/internal/processors/platform/PlatformTarget;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
             JniMethod M_PLATFORM_PROCESSOR_EXTENSIONS = JniMethod("extensions", "()Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
             JniMethod M_PLATFORM_PROCESSOR_ATOMIC_LONG = JniMethod("atomicLong", "(Ljava/lang/String;JZ)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
+            JniMethod M_PLATFORM_PROCESSOR_GET_IGNITE_CONFIGURATION = JniMethod("getIgniteConfiguration", "(J)V", false);
 
             const char* C_PLATFORM_TARGET = "org/apache/ignite/internal/processors/platform/PlatformTarget";
             JniMethod M_PLATFORM_TARGET_IN_STREAM_OUT_LONG = JniMethod("inStreamOutLong", "(IJ)J", false);
@@ -636,6 +639,8 @@ namespace ignite
                 m_PlatformProcessor_cache = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_CACHE);
                 m_PlatformProcessor_createCache = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_CREATE_CACHE);
                 m_PlatformProcessor_getOrCreateCache = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_GET_OR_CREATE_CACHE);
+                m_PlatformProcessor_createCacheFromConfig = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_CREATE_CACHE_FROM_CONFIG);
+                m_PlatformProcessor_getOrCreateCacheFromConfig = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_GET_OR_CREATE_CACHE_FROM_CONFIG);
                 m_PlatformProcessor_destroyCache = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_DESTROY_CACHE);
                 m_PlatformProcessor_affinity = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_AFFINITY);
                 m_PlatformProcessor_dataStreamer = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_DATA_STREAMER);
@@ -647,6 +652,7 @@ namespace ignite
                 m_PlatformProcessor_services = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_SERVICES);
                 m_PlatformProcessor_extensions = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_EXTENSIONS);
                 m_PlatformProcessor_atomicLong = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_ATOMIC_LONG);
+				m_PlatformProcessor_getIgniteConfiguration = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_GET_IGNITE_CONFIGURATION);
 
                 c_PlatformTarget = FindClass(env, C_PLATFORM_TARGET);
                 m_PlatformTarget_inStreamOutLong = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_STREAM_OUT_LONG);
@@ -1120,6 +1126,17 @@ namespace ignite
                 return LocalToGlobal(env, cache);
             }
 
+            jobject JniContext::ProcessorCacheFromConfig0(jobject obj, long memPtr, jmethodID mthd, JniErrorInfo* errInfo)
+            {
+                JNIEnv* env = Attach();
+
+                jobject cache = env->CallObjectMethod(obj, mthd, memPtr);
+
+                ExceptionCheck(env, errInfo);
+
+                return LocalToGlobal(env, cache);
+            }
+
             jobject JniContext::ProcessorCache(jobject obj, const char* name) {
                 return ProcessorCache(obj, name, NULL);
             }
@@ -1164,6 +1181,24 @@ namespace ignite
                 ExceptionCheck(env, errInfo);
             }
 
+            jobject JniContext::ProcessorCreateCacheFromConfig(jobject obj, long memPtr) {
+                return ProcessorCreateCacheFromConfig(obj, memPtr, NULL);
+            }
+
+            jobject JniContext::ProcessorCreateCacheFromConfig(jobject obj, long memPtr, JniErrorInfo* errInfo)
+            {
+                return ProcessorCacheFromConfig0(obj, memPtr, jvm->GetMembers().m_PlatformProcessor_createCacheFromConfig, errInfo);
+            }
+
+            jobject JniContext::ProcessorGetOrCreateCacheFromConfig(jobject obj, long memPtr) {
+                return ProcessorGetOrCreateCacheFromConfig(obj, memPtr, NULL);
+            }
+
+            jobject JniContext::ProcessorGetOrCreateCacheFromConfig(jobject obj, long memPtr, JniErrorInfo* errInfo)
+            {
+                return ProcessorCacheFromConfig0(obj, memPtr, jvm->GetMembers().m_PlatformProcessor_getOrCreateCacheFromConfig, errInfo);
+            }
+
             jobject JniContext::ProcessorAffinity(jobject obj, const char* name) {
                 JNIEnv* env = Attach();
 
@@ -1272,6 +1307,15 @@ namespace ignite
                 return LocalToGlobal(env, res);
             }
 
+            void JniContext::ProcessorGetIgniteConfiguration(jobject obj, long memPtr)
+            {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformProcessor_getIgniteConfiguration, memPtr);
+
+                ExceptionCheck(env);
+            }
+
             long long JniContext::TargetInStreamOutLong(jobject obj, int opType, long long memPtr, JniErrorInfo* err) {
                 JNIEnv* env = Attach();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
index 5a1e176..481adfb 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
@@ -60,6 +60,7 @@
     <Reference Include="System.XML" />
   </ItemGroup>
   <ItemGroup>
+    <Compile Include="Cache\CacheConfigurationTest.cs" />
     <Compile Include="Cache\CacheDynamicStartTest.cs" />
     <Compile Include="Cache\CacheTestAsyncWrapper.cs" />
     <Compile Include="Cache\CacheAbstractTest.cs" />
@@ -74,6 +75,7 @@
     <Compile Include="Cache\CachePartitionedTest.cs" />
     <Compile Include="Cache\CacheReplicatedAtomicTest.cs" />
     <Compile Include="Cache\CacheReplicatedTest.cs" />
+    <Compile Include="Cache\Query\CacheQueriesCodeConfigurationTest.cs" />
     <Compile Include="Cache\Query\Continuous\ContinuousQueryAbstractTest.cs" />
     <Compile Include="Cache\Query\Continuous\ContinuousQueryAtomicBackupTest.cs" />
     <Compile Include="Cache\Query\Continuous\ContinuousQueryAtomicNoBackupTest.cs" />
@@ -113,6 +115,7 @@
     <Compile Include="ExceptionsTest.cs" />
     <Compile Include="ExecutableTest.cs" />
     <Compile Include="FutureTest.cs" />
+    <Compile Include="IgniteConfigurationTest.cs" />
     <Compile Include="IgniteTestBase.cs" />
     <Compile Include="LifecycleTest.cs" />
     <Compile Include="LoadDllTest.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
index 9232665..44db6f7 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
@@ -1341,6 +1341,15 @@ namespace Apache.Ignite.Core.Tests.Binary
             }
         }
 
+        [Test]
+        public void TestBinaryConfigurationValidation()
+        {
+            var cfg = new BinaryConfiguration(typeof (PropertyType)) {Types = new[] {"PropertyType"}};
+
+            // ReSharper disable once ObjectCreationAsStatement
+            Assert.Throws<BinaryObjectException>(() => new Marshaller(cfg));
+        }
+
         private static void CheckKeepSerialized(BinaryConfiguration cfg, bool expKeep)
         {
             if (cfg.TypeConfigurations == null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
index e57df6f..9f0528c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
@@ -292,7 +292,7 @@ namespace Apache.Ignite.Core.Tests.Cache
         public virtual void StartGrids() {
             TestUtils.KillProcesses();
 
-            IgniteConfigurationEx cfg = new IgniteConfigurationEx();
+            IgniteConfiguration cfg = new IgniteConfiguration();
 
             BinaryConfiguration portCfg = new BinaryConfiguration();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAffinityTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAffinityTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAffinityTest.cs
index 5a1af03..689804c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAffinityTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAffinityTest.cs
@@ -36,7 +36,7 @@ namespace Apache.Ignite.Core.Tests.Cache
         {
             TestUtils.KillProcesses();
 
-            IgniteConfigurationEx cfg = new IgniteConfigurationEx();
+            IgniteConfiguration cfg = new IgniteConfiguration();
 
             cfg.JvmClasspath = TestUtils.CreateTestClasspath();
             cfg.JvmOptions = TestUtils.TestJavaOptions();