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

[09/14] ignite git commit: IGNITE-1513: Merged Java to core module.

http://git-wip-us.apache.org/repos/asf/ignite/blob/8045c820/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
new file mode 100644
index 0000000..bfa4570
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
@@ -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.
+ */
+
+package org.apache.ignite.internal.processors.platform.services;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteServices;
+import org.apache.ignite.internal.portable.PortableRawReaderEx;
+import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
+import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetService;
+import org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetServiceImpl;
+import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
+import org.apache.ignite.internal.processors.platform.utils.PlatformWriterBiClosure;
+import org.apache.ignite.internal.processors.platform.utils.PlatformWriterClosure;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.services.Service;
+import org.apache.ignite.services.ServiceConfiguration;
+import org.apache.ignite.services.ServiceDescriptor;
+
+/**
+ * Interop services.
+ */
+@SuppressWarnings({"UnusedDeclaration"})
+public class PlatformServices extends PlatformAbstractTarget {
+    /** */
+    private static final int OP_DOTNET_DEPLOY = 1;
+
+    /** */
+    private static final int OP_DOTNET_DEPLOY_MULTIPLE = 2;
+
+    /** */
+    private static final int OP_DOTNET_SERVICES = 3;
+
+    /** */
+    private static final int OP_DOTNET_INVOKE = 4;
+
+    /** */
+    private static final int OP_DESCRIPTORS = 5;
+
+    /** */
+    private final IgniteServices services;
+
+    /** Server keep portable flag. */
+    private final boolean srvKeepPortable;
+
+    /**
+     * Ctor.
+     *
+     * @param platformCtx Context.
+     * @param services Services facade.
+     * @param srvKeepPortable Server keep portable flag.
+     */
+    public PlatformServices(PlatformContext platformCtx, IgniteServices services, boolean srvKeepPortable) {
+        super(platformCtx);
+
+        assert services != null;
+
+        this.services = services;
+        this.srvKeepPortable = srvKeepPortable;
+    }
+
+    /**
+     * Gets services with asynchronous mode enabled.
+     *
+     * @return Services with asynchronous mode enabled.
+     */
+    public PlatformServices withAsync() {
+        if (services.isAsync())
+            return this;
+
+        return new PlatformServices(platformCtx, services.withAsync(), srvKeepPortable);
+    }
+
+    /**
+     * Gets services with server "keep portable" mode enabled.
+     *
+     * @return Services with server "keep portable" mode enabled.
+     */
+    public PlatformServices withServerKeepPortable() {
+        return srvKeepPortable ? this : new PlatformServices(platformCtx, services, true);
+    }
+
+    /**
+     * Cancels service deployment.
+     *
+     * @param name Name of service to cancel.
+     */
+    public void cancel(String name) {
+        services.cancel(name);
+    }
+
+    /**
+     * Cancels all deployed services.
+     */
+    public void cancelAll() {
+        services.cancelAll();
+    }
+
+    /**
+     * Gets a remote handle on the service.
+     *
+     * @param name Service name.
+     * @param sticky Whether or not Ignite should always contact the same remote service.
+     * @return Either proxy over remote service or local service if it is deployed locally.
+     */
+    public Object dotNetServiceProxy(String name, boolean sticky) {
+        return services.serviceProxy(name, PlatformDotNetService.class, sticky);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long processInStreamOutLong(int type, PortableRawReaderEx reader)
+        throws IgniteCheckedException {
+        switch (type) {
+            case OP_DOTNET_DEPLOY: {
+                ServiceConfiguration cfg = new ServiceConfiguration();
+
+                cfg.setName(reader.readString());
+                cfg.setService(new PlatformDotNetServiceImpl(reader.readObjectDetached(), platformCtx, srvKeepPortable));
+                cfg.setTotalCount(reader.readInt());
+                cfg.setMaxPerNodeCount(reader.readInt());
+                cfg.setCacheName(reader.readString());
+                cfg.setAffinityKey(reader.readObjectDetached());
+
+                Object filter = reader.readObjectDetached();
+
+                if (filter != null)
+                    cfg.setNodeFilter(platformCtx.createClusterNodeFilter(filter));
+
+                services.deploy(cfg);
+
+                return TRUE;
+            }
+
+            case OP_DOTNET_DEPLOY_MULTIPLE: {
+                String name = reader.readString();
+                Object svc = reader.readObjectDetached();
+                int totalCnt = reader.readInt();
+                int maxPerNodeCnt = reader.readInt();
+
+                services.deployMultiple(name, new PlatformDotNetServiceImpl(svc, platformCtx, srvKeepPortable),
+                    totalCnt, maxPerNodeCnt);
+
+                return TRUE;
+            }
+
+            default:
+                return super.processInStreamOutLong(type, reader);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void processInStreamOutStream(int type, PortableRawReaderEx reader, PortableRawWriterEx writer)
+        throws IgniteCheckedException {
+        switch (type) {
+            case OP_DOTNET_SERVICES: {
+                Collection<Service> svcs = services.services(reader.readString());
+
+                PlatformUtils.writeNullableCollection(writer, svcs,
+                    new PlatformWriterClosure<Service>() {
+                        @Override public void write(PortableRawWriterEx writer, Service svc) {
+                            writer.writeLong(((PlatformService) svc).pointer());
+                        }
+                    },
+                    new IgnitePredicate<Service>() {
+                        @Override public boolean apply(Service svc) {
+                            return svc instanceof PlatformDotNetService;
+                        }
+                    }
+                );
+
+                return;
+            }
+
+            default:
+                super.processInStreamOutStream(type, reader, writer);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void processInObjectStreamOutStream(int type, Object arg, PortableRawReaderEx reader,
+        PortableRawWriterEx writer) throws IgniteCheckedException {
+        switch (type) {
+            case OP_DOTNET_INVOKE: {
+                assert arg != null;
+                assert arg instanceof PlatformDotNetService;
+
+                String mthdName = reader.readString();
+
+                Object[] args;
+
+                if (reader.readBoolean()) {
+                    args = new Object[reader.readInt()];
+
+                    for (int i = 0; i < args.length; i++)
+                        args[i] = reader.readObjectDetached();
+                }
+                else
+                    args = null;
+
+                try {
+                    Object result = ((PlatformDotNetService)arg).invokeMethod(mthdName, srvKeepPortable, args);
+
+                    PlatformUtils.writeInvocationResult(writer, result, null);
+                }
+                catch (Exception e) {
+                    PlatformUtils.writeInvocationResult(writer, null, e);
+                }
+
+                return;
+            }
+
+            default:
+                super.processInObjectStreamOutStream(type, arg, reader, writer);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void processOutStream(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
+        switch (type) {
+            case OP_DESCRIPTORS: {
+                Collection<ServiceDescriptor> descs = services.serviceDescriptors();
+
+                PlatformUtils.writeCollection(writer, descs, new PlatformWriterClosure<ServiceDescriptor>() {
+                    @Override public void write(PortableRawWriterEx writer, ServiceDescriptor d) {
+                        writer.writeString(d.name());
+                        writer.writeString(d.cacheName());
+                        writer.writeInt(d.maxPerNodeCount());
+                        writer.writeInt(d.totalCount());
+                        writer.writeUuid(d.originNodeId());
+                        writer.writeObject(d.affinityKey());
+
+                        Map<UUID, Integer> top = d.topologySnapshot();
+
+                        PlatformUtils.writeMap(writer, top, new PlatformWriterBiClosure<UUID, Integer>() {
+                            @Override public void write(PortableRawWriterEx writer, UUID key, Integer val) {
+                                writer.writeUuid(key);
+                                writer.writeInt(val);
+                            }
+                        });
+                    }
+                });
+
+                return;
+            }
+
+            default:
+                super.processOutStream(type, writer);
+        }
+    }
+
+    /** <inheritDoc /> */
+    @Override protected IgniteFuture currentFuture() throws IgniteCheckedException {
+        return services.future();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8045c820/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
new file mode 100644
index 0000000..1d2c315
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
@@ -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.
+ */
+
+package org.apache.ignite.internal.processors.platform.transactions;
+
+import java.util.Date;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteTransactions;
+import org.apache.ignite.configuration.TransactionConfiguration;
+import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
+import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils;
+import org.apache.ignite.internal.util.GridConcurrentFactory;
+import org.apache.ignite.internal.util.typedef.C1;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+import org.apache.ignite.transactions.TransactionMetrics;
+
+/**
+ * Native transaction wrapper implementation.
+ */
+@SuppressWarnings({"unchecked", "UnusedDeclaration", "TryFinallyCanBeTryWithResources"})
+public class PlatformTransactions extends PlatformAbstractTarget {
+    /** */
+    public static final int OP_CACHE_CONFIG_PARAMETERS = 1;
+
+    /** */
+    public static final int OP_METRICS = 2;
+
+    /** */
+    private final IgniteTransactions txs;
+
+    /** Map with currently active transactions. */
+    private final ConcurrentMap<Long, Transaction> txMap = GridConcurrentFactory.newMap();
+
+    /** Transaction ID sequence. Must be static to ensure uniqueness across different caches. */
+    private static final AtomicLong TX_ID_GEN = new AtomicLong();
+
+    /**
+     * Constructor.
+     *
+     * @param platformCtx Context.
+     */
+    public PlatformTransactions(PlatformContext platformCtx) {
+        super(platformCtx);
+
+        txs = platformCtx.kernalContext().grid().transactions();
+    }
+
+    /**
+     * @param concurrency Concurrency.
+     * @param isolation Isolation.
+     * @param timeout Timeout
+     * @param txSize Number of entries participating in transaction.
+     * @return Transaction thread ID.
+     */
+    public long txStart(int concurrency, int isolation, long timeout, int txSize) {
+        TransactionConcurrency txConcurrency = TransactionConcurrency.fromOrdinal(concurrency);
+
+        assert txConcurrency != null;
+
+        TransactionIsolation txIsolation = TransactionIsolation.fromOrdinal(isolation);
+
+        assert txIsolation != null;
+
+        Transaction tx = txs.txStart(txConcurrency, txIsolation);
+
+        return registerTx(tx);
+    }
+
+    /**
+     * @param id Transaction ID.
+     * @throws org.apache.ignite.IgniteCheckedException In case of error.
+     */
+    public int txCommit(long id) throws IgniteCheckedException {
+        tx(id).commit();
+
+        return txClose(id);
+    }
+
+    /**
+     * @param id Transaction ID.
+     * @throws org.apache.ignite.IgniteCheckedException In case of error.
+     */
+    public int txRollback(long id) throws IgniteCheckedException {
+        tx(id).rollback();
+
+        return txClose(id);
+    }
+
+    /**
+     * @param id Transaction ID.
+     * @throws org.apache.ignite.IgniteCheckedException In case of error.
+     * @return Transaction state.
+     */
+    public int txClose(long id) throws IgniteCheckedException {
+        Transaction tx = tx(id);
+
+        try {
+            tx.close();
+
+            return tx.state().ordinal();
+        }
+        finally {
+            unregisterTx(id);
+        }
+    }
+
+    /**
+     * @param id Transaction ID.
+     * @return Transaction state.
+     */
+    public int txState(long id) {
+        Transaction tx = tx(id);
+
+        return tx.state().ordinal();
+    }
+
+    /**
+     * @param id Transaction ID.
+     * @return {@code True} if rollback only flag was set.
+     */
+    public boolean txSetRollbackOnly(long id) {
+        Transaction tx = tx(id);
+
+        return tx.setRollbackOnly();
+    }
+
+    /**
+     * Commits tx in async mode.
+     */
+    public void txCommitAsync(final long txId, final long futId) {
+        final Transaction asyncTx = (Transaction)tx(txId).withAsync();
+
+        asyncTx.commit();
+
+        listenAndNotifyIntFuture(futId, asyncTx);
+    }
+
+    /**
+     * Rolls back tx in async mode.
+     */
+    public void txRollbackAsync(final long txId, final long futId) {
+        final Transaction asyncTx = (Transaction)tx(txId).withAsync();
+
+        asyncTx.rollback();
+
+        listenAndNotifyIntFuture(futId, asyncTx);
+    }
+
+    /**
+     * Listens to the transaction future and notifies .NET int future.
+     */
+    private void listenAndNotifyIntFuture(final long futId, final Transaction asyncTx) {
+        IgniteFuture fut = asyncTx.future().chain(new C1<IgniteFuture, Object>() {
+            private static final long serialVersionUID = 0L;
+
+            @Override public Object apply(IgniteFuture fut) {
+                return null;
+            }
+        });
+
+        PlatformFutureUtils.listen(platformCtx, fut, futId, PlatformFutureUtils.TYP_OBJ, this);
+    }
+
+    /**
+     * Resets transaction metrics.
+     */
+    public void resetMetrics() {
+       txs.resetMetrics();
+    }
+
+    /**
+     * Register transaction.
+     *
+     * @param tx Transaction.
+     * @return Transaction ID.
+     */
+    private long registerTx(Transaction tx) {
+        long id = TX_ID_GEN.incrementAndGet();
+
+        Transaction old = txMap.put(id, tx);
+
+        assert old == null : "Duplicate TX ids: " + old;
+
+        return id;
+    }
+
+    /**
+     * Unregister transaction.
+     *
+     * @param id Transaction ID.
+     */
+    private void unregisterTx(long id) {
+        Transaction tx = txMap.remove(id);
+
+        assert tx != null : "Failed to unregister transaction: " + id;
+    }
+
+    /**
+     * Get transaction by ID.
+     *
+     * @param id ID.
+     * @return Transaction.
+     */
+    private Transaction tx(long id) {
+        Transaction tx = txMap.get(id);
+
+        assert tx != null : "Transaction not found for ID: " + id;
+
+        return tx;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void processOutStream(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
+        switch (type) {
+            case OP_CACHE_CONFIG_PARAMETERS:
+                TransactionConfiguration txCfg = platformCtx.kernalContext().config().getTransactionConfiguration();
+
+                writer.writeEnum(txCfg.getDefaultTxConcurrency());
+                writer.writeEnum(txCfg.getDefaultTxIsolation());
+                writer.writeLong(txCfg.getDefaultTxTimeout());
+
+                break;
+
+            case OP_METRICS:
+                TransactionMetrics metrics = txs.metrics();
+
+                writer.writeDate(new Date(metrics.commitTime()));
+                writer.writeDate(new Date(metrics.rollbackTime()));
+                writer.writeInt(metrics.txCommits());
+                writer.writeInt(metrics.txRollbacks());
+
+                break;
+
+            default:
+                super.processOutStream(type, writer);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8045c820/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java
new file mode 100644
index 0000000..0019986
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java
@@ -0,0 +1,397 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
+import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.callback.PlatformCallbackGateway;
+import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
+import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
+import org.apache.ignite.lang.IgniteBiInClosure;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Interop future utils.
+ */
+public class PlatformFutureUtils {
+    /** Future type: byte. */
+    public static final int TYP_BYTE = 1;
+
+    /** Future type: boolean. */
+    public static final int TYP_BOOL = 2;
+
+    /** Future type: short. */
+    public static final int TYP_SHORT = 3;
+
+    /** Future type: char. */
+    public static final int TYP_CHAR = 4;
+
+    /** Future type: int. */
+    public static final int TYP_INT = 5;
+
+    /** Future type: float. */
+    public static final int TYP_FLOAT = 6;
+
+    /** Future type: long. */
+    public static final int TYP_LONG = 7;
+
+    /** Future type: double. */
+    public static final int TYP_DOUBLE = 8;
+
+    /** Future type: object. */
+    public static final int TYP_OBJ = 9;
+
+    /**
+     * Listen future.
+     *
+     * @param ctx Context.
+     * @param fut Java future.
+     * @param futPtr Native future pointer.
+     * @param typ Expected return type.
+     */
+    public static void listen(final PlatformContext ctx, IgniteInternalFuture fut, final long futPtr, final int typ,
+        PlatformAbstractTarget target) {
+        listen(ctx, new InternalFutureListenable(fut), futPtr, typ, null, target);
+    }
+    /**
+     * Listen future.
+     *
+     * @param ctx Context.
+     * @param fut Java future.
+     * @param futPtr Native future pointer.
+     * @param typ Expected return type.
+     */
+    public static void listen(final PlatformContext ctx, IgniteFuture fut, final long futPtr, final int typ,
+        PlatformAbstractTarget target) {
+        listen(ctx, new FutureListenable(fut), futPtr, typ, null, target);
+    }
+
+    /**
+     * Listen future.
+     *
+     * @param ctx Context.
+     * @param fut Java future.
+     * @param futPtr Native future pointer.
+     * @param typ Expected return type.
+     * @param writer Writer.
+     */
+    public static void listen(final PlatformContext ctx, IgniteInternalFuture fut, final long futPtr, final int typ,
+        Writer writer, PlatformAbstractTarget target) {
+        listen(ctx, new InternalFutureListenable(fut), futPtr, typ, writer, target);
+    }
+
+    /**
+     * Listen future.
+     *
+     * @param ctx Context.
+     * @param fut Java future.
+     * @param futPtr Native future pointer.
+     * @param typ Expected return type.
+     * @param writer Writer.
+     */
+    public static void listen(final PlatformContext ctx, IgniteFuture fut, final long futPtr, final int typ,
+        Writer writer, PlatformAbstractTarget target) {
+        listen(ctx, new FutureListenable(fut), futPtr, typ, writer, target);
+    }
+
+    /**
+     * Listen future.
+     *
+     * @param ctx Context.
+     * @param fut Java future.
+     * @param futPtr Native future pointer.
+     * @param writer Writer.
+     */
+    public static void listen(final PlatformContext ctx, IgniteInternalFuture fut, final long futPtr, Writer writer,
+        PlatformAbstractTarget target) {
+        listen(ctx, new InternalFutureListenable(fut), futPtr, TYP_OBJ, writer, target);
+    }
+
+    /**
+     * Listen future.
+     *
+     * @param ctx Context.
+     * @param listenable Listenable entry.
+     * @param futPtr Native future pointer.
+     * @param typ Expected return type.
+     * @param writer Optional writer.
+     */
+    @SuppressWarnings("unchecked")
+    private static void listen(final PlatformContext ctx, Listenable listenable, final long futPtr, final int typ,
+        @Nullable final Writer writer, final PlatformAbstractTarget target) {
+        final PlatformCallbackGateway gate = ctx.gateway();
+
+        listenable.listen(new IgniteBiInClosure<Object, Throwable>() {
+            private static final long serialVersionUID = 0L;
+
+            @Override public void apply(Object res, Throwable err) {
+                if (err instanceof Exception)
+                    err = target.convertException((Exception)err);
+
+                if (writer != null && writeToWriter(res, err, ctx, writer, futPtr))
+                    return;
+
+                if (err != null) {
+                    writeFutureError(ctx, futPtr, err);
+
+                    return;
+                }
+
+                try {
+                    if (typ == TYP_OBJ) {
+                        if (res == null)
+                            gate.futureNullResult(futPtr);
+                        else {
+                            try (PlatformMemory mem = ctx.memory().allocate()) {
+                                PlatformOutputStream out = mem.output();
+
+                                PortableRawWriterEx outWriter = ctx.writer(out);
+
+                                outWriter.writeObjectDetached(res);
+
+                                out.synchronize();
+
+                                gate.futureObjectResult(futPtr, mem.pointer());
+                            }
+                        }
+                    }
+                    else if (res == null)
+                        gate.futureNullResult(futPtr);
+                    else {
+                        switch (typ) {
+                            case TYP_BYTE:
+                                gate.futureByteResult(futPtr, (byte) res);
+
+                                break;
+
+                            case TYP_BOOL:
+                                gate.futureBoolResult(futPtr, (boolean) res ? 1 : 0);
+
+                                break;
+
+                            case TYP_SHORT:
+                                gate.futureShortResult(futPtr, (short) res);
+
+                                break;
+
+                            case TYP_CHAR:
+                                gate.futureCharResult(futPtr, (char) res);
+
+                                break;
+
+                            case TYP_INT:
+                                gate.futureIntResult(futPtr, (int) res);
+
+                                break;
+
+                            case TYP_FLOAT:
+                                gate.futureFloatResult(futPtr, (float) res);
+
+                                break;
+
+                            case TYP_LONG:
+                                gate.futureLongResult(futPtr, (long) res);
+
+                                break;
+
+                            case TYP_DOUBLE:
+                                gate.futureDoubleResult(futPtr, (double) res);
+
+                                break;
+
+                            default:
+                                assert false : "Should not reach this: " + typ;
+                        }
+                    }
+                }
+                catch (Throwable t) {
+                    writeFutureError(ctx, futPtr, t);
+
+                    if (t instanceof Error)
+                        throw t;
+                }
+            }
+        });
+    }
+
+    /**
+     * Write future error.
+     *
+     * @param ctx Context.
+     * @param futPtr Future pointer.
+     * @param err Error.
+     */
+    private static void writeFutureError(final PlatformContext ctx, long futPtr, Throwable err) {
+        try (PlatformMemory mem = ctx.memory().allocate()) {
+            PlatformOutputStream out = mem.output();
+
+            PortableRawWriterEx outWriter = ctx.writer(out);
+
+            outWriter.writeString(err.getClass().getName());
+            outWriter.writeString(err.getMessage());
+
+            PlatformUtils.writeErrorData(err, outWriter);
+
+            out.synchronize();
+
+            ctx.gateway().futureError(futPtr, mem.pointer());
+        }
+    }
+
+    /**
+     * Write result to a custom writer
+     *
+     * @param obj Object to write.
+     * @param err Error to write.
+     * @param ctx Context.
+     * @param writer Writer.
+     * @param futPtr Future pointer.
+     * @return Value indicating whether custom write was performed. When false, default write will be used.
+     */
+    private static boolean writeToWriter(Object obj, Throwable err, PlatformContext ctx, Writer writer, long futPtr) {
+        boolean canWrite = writer.canWrite(obj, err);
+
+        if (!canWrite)
+            return false;
+
+        try (PlatformMemory mem = ctx.memory().allocate()) {
+            PlatformOutputStream out = mem.output();
+
+            PortableRawWriterEx outWriter = ctx.writer(out);
+
+            writer.write(outWriter, obj, err);
+
+            out.synchronize();
+
+            ctx.gateway().futureObjectResult(futPtr, mem.pointer());
+        }
+
+        return true;
+    }
+
+    /**
+     * Writer allowing special future result handling.
+     */
+    public static interface Writer {
+        /**
+         * Write object.
+         *
+         * @param writer Writer.
+         * @param obj Object.
+         * @param err Error.
+         */
+        public void write(PortableRawWriterEx writer, Object obj, Throwable err);
+
+        /**
+         * Determines whether this writer can write given data.
+         *
+         * @param obj Object.
+         * @param err Error.
+         * @return Value indicating whether this writer can write given data.
+         */
+        public boolean canWrite(Object obj, Throwable err);
+    }
+
+    /**
+     * Listenable entry.
+     */
+    private static interface Listenable {
+        /**
+         * Listen.
+         *
+         * @param lsnr Listener.
+         */
+        public void listen(IgniteBiInClosure<Object, Throwable> lsnr);
+    }
+
+    /**
+     * Listenable around Ignite future.
+     */
+    private static class FutureListenable implements Listenable {
+        /** Future. */
+        private final IgniteFuture fut;
+
+        /**
+         * Constructor.
+         *
+         * @param fut Future.
+         */
+        public FutureListenable(IgniteFuture fut) {
+            this.fut = fut;
+        }
+
+        /** {@inheritDoc} */
+        @SuppressWarnings("unchecked")
+        @Override public void listen(final IgniteBiInClosure<Object, Throwable> lsnr) {
+            fut.listen(new IgniteInClosure<IgniteFuture>() {
+                private static final long serialVersionUID = 0L;
+
+                @Override public void apply(IgniteFuture fut0) {
+                    try {
+                        lsnr.apply(fut0.get(), null);
+                    }
+                    catch (Throwable err) {
+                        lsnr.apply(null, err);
+
+                        if (err instanceof Error)
+                            throw err;
+                    }
+                }
+            });
+        }
+    }
+
+    /**
+     * Listenable around Ignite future.
+     */
+    private static class InternalFutureListenable implements Listenable {
+        /** Future. */
+        private final IgniteInternalFuture fut;
+
+        /**
+         * Constructor.
+         *
+         * @param fut Future.
+         */
+        public InternalFutureListenable(IgniteInternalFuture fut) {
+            this.fut = fut;
+        }
+
+        /** {@inheritDoc} */
+        @SuppressWarnings("unchecked")
+        @Override public void listen(final IgniteBiInClosure<Object, Throwable> lsnr) {
+            fut.listen(new IgniteInClosure<IgniteInternalFuture>() {
+                private static final long serialVersionUID = 0L;
+
+                @Override public void apply(IgniteInternalFuture fut0) {
+                    try {
+                        lsnr.apply(fut0.get(), null);
+                    }
+                    catch (Throwable err) {
+                        lsnr.apply(null, err);
+                    }
+                }
+            });
+        }
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8045c820/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformReaderBiClosure.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformReaderBiClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformReaderBiClosure.java
new file mode 100644
index 0000000..79759e9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformReaderBiClosure.java
@@ -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.
+ */
+
+package org.apache.ignite.internal.processors.platform.utils;
+
+import org.apache.ignite.internal.portable.PortableRawReaderEx;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/**
+ * Reader bi-closure.
+ */
+public interface PlatformReaderBiClosure<T1, T2> {
+    /**
+     * Read object from reader.
+     *
+     * @param reader Reader.
+     * @return Object.
+     */
+    IgniteBiTuple<T1, T2> read(PortableRawReaderEx reader);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8045c820/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformReaderClosure.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformReaderClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformReaderClosure.java
new file mode 100644
index 0000000..2d9b44a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformReaderClosure.java
@@ -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.
+ */
+
+package org.apache.ignite.internal.processors.platform.utils;
+
+import org.apache.ignite.internal.portable.PortableRawReaderEx;
+
+/**
+ * Reader closure.
+ */
+public interface PlatformReaderClosure<T> {
+
+    /**
+     * Read object from reader.
+     *
+     * @param reader Reader.
+     * @return Object.
+     */
+    T read(PortableRawReaderEx reader);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8045c820/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
new file mode 100644
index 0000000..11d8371
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java
@@ -0,0 +1,812 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.Ignite;
+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.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.portable.PortableRawReaderEx;
+import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.PlatformExtendedException;
+import org.apache.ignite.internal.processors.platform.PlatformNativeException;
+import org.apache.ignite.internal.processors.platform.PlatformProcessor;
+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.PlatformMemoryUtils;
+import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
+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.PlatformDotNetPortableConfiguration;
+import org.apache.ignite.platform.dotnet.PlatformDotNetPortableTypeConfiguration;
+import org.jetbrains.annotations.Nullable;
+
+import javax.cache.CacheException;
+import javax.cache.event.CacheEntryEvent;
+import javax.cache.event.CacheEntryListenerException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PREFIX;
+
+/**
+ * Platform utility methods.
+ */
+@SuppressWarnings({"UnusedDeclaration", "unchecked"})
+public class PlatformUtils {
+    /** Node attribute: platform. */
+    public static final String ATTR_PLATFORM = ATTR_PREFIX  + ".platform";
+
+    /** Platform: CPP. */
+    public static final String PLATFORM_CPP = "cpp";
+
+    /** Platform: .Net. */
+    public static final String PLATFORM_DOTNET = "dotnet";
+
+    /** Operation: prepare .Net platform. */
+    public static final int OP_PREPARE_DOT_NET = 1;
+
+    /** Amount of peek modes available. */
+    private static final int CACHE_PEEK_MODES_CNT = CachePeekMode.values().length;
+
+    /** Cache peek modes. */
+    private static volatile CachePeekMode[][] CACHE_PEEK_MODES;
+
+    /**
+     * Static initializer.
+     */
+    static {
+        int len = 1 << CACHE_PEEK_MODES_CNT;
+
+        synchronized (PlatformUtils.class) {
+            CACHE_PEEK_MODES = new CachePeekMode[len][];
+
+            CACHE_PEEK_MODES[0] = new CachePeekMode[0];
+        }
+    }
+
+    /**
+     * Write nullable collection to the writer.
+     *
+     * @param writer Writer.
+     * @param col Collection to write.
+     */
+    public static <T> void writeNullableCollection(PortableRawWriterEx writer, @Nullable Collection<T> col) {
+        writeNullableCollection(writer, col, null, null);
+    }
+
+    /**
+     * Write nullable collection to the writer.
+     *
+     * @param writer Writer.
+     * @param col Collection to write.
+     * @param writeClo Writer closure.
+     */
+    public static <T> void writeNullableCollection(PortableRawWriterEx writer, @Nullable Collection<T> col,
+        @Nullable PlatformWriterClosure<T> writeClo) {
+        writeNullableCollection(writer, col, writeClo, null);
+    }
+
+    /**
+     * Write collection to the writer.
+     *
+     * @param writer Writer.
+     * @param col Collection to write.
+     * @param writeClo Optional writer closure.
+     * @param filter Optional filter.
+     */
+    public static <T> void writeNullableCollection(PortableRawWriterEx writer, @Nullable Collection<T> col,
+        @Nullable PlatformWriterClosure<T> writeClo, @Nullable IgnitePredicate<T> filter) {
+        if (col != null) {
+            writer.writeBoolean(true);
+
+            writeCollection(writer, col, writeClo, filter);
+        }
+        else
+            writer.writeBoolean(false);
+    }
+
+    /**
+     * Write collection to the writer.
+     *
+     * @param writer Writer.
+     * @param col Collection to write.
+     */
+    public static <T> void writeCollection(PortableRawWriterEx writer, Collection<T> col) {
+        writeCollection(writer, col, null, null);
+    }
+
+    /**
+     * Write collection to the writer.
+     *
+     * @param writer Writer.
+     * @param col Collection to write.
+     * @param writeClo Writer closure.
+     */
+    public static <T> void writeCollection(PortableRawWriterEx writer, Collection<T> col,
+        @Nullable PlatformWriterClosure<T> writeClo) {
+        writeCollection(writer, col, writeClo, null);
+    }
+
+    /**
+     * Write collection to the writer.
+     *
+     * @param writer Writer.
+     * @param col Collection to write.
+     * @param writeClo Optional writer closure.
+     * @param filter Optional filter.
+     */
+    public static <T> void writeCollection(PortableRawWriterEx writer, Collection<T> col,
+        @Nullable PlatformWriterClosure<T> writeClo, @Nullable IgnitePredicate<T> filter) {
+        assert col != null;
+
+        if (filter == null) {
+            writer.writeInt(col.size());
+
+            if (writeClo == null) {
+                for (T entry : col)
+                    writer.writeObject(entry);
+            }
+            else {
+                for (T entry : col)
+                    writeClo.write(writer, entry);
+            }
+        }
+        else {
+            int pos = writer.reserveInt();
+            int cnt = 0;
+
+            for (T entry : col) {
+                if (filter.apply(entry)) {
+                    cnt++;
+
+                    if (writeClo == null)
+                        writer.writeObject(entry);
+                    else
+                        writeClo.write(writer, entry);
+                }
+            }
+
+            writer.writeInt(pos, cnt);
+        }
+    }
+
+    /**
+     * Write nullable map to the writer.
+     *
+     * @param writer Writer.
+     * @param map Map to write.
+     */
+    public static <K, V> void writeNullableMap(PortableRawWriterEx writer, @Nullable Map<K, V> map) {
+        if (map != null) {
+            writer.writeBoolean(true);
+
+            writeMap(writer, map);
+        }
+        else
+            writer.writeBoolean(false);
+    }
+
+    /**
+     * Write nullable map to the writer.
+     *
+     * @param writer Writer.
+     * @param map Map to write.
+     */
+    public static <K, V> void writeMap(PortableRawWriterEx writer, Map<K, V> map) {
+        assert map != null;
+
+        writeMap(writer, map, null);
+    }
+
+    /**
+     * Write nullable map to the writer.
+     *
+     * @param writer Writer.
+     * @param map Map to write.
+     * @param writeClo Writer closure.
+     */
+    public static <K, V> void writeMap(PortableRawWriterEx writer, Map<K, V> map,
+        @Nullable PlatformWriterBiClosure<K, V> writeClo) {
+        assert map != null;
+
+        writer.writeInt(map.size());
+
+        if (writeClo == null) {
+            for (Map.Entry<K, V> entry : map.entrySet()) {
+                writer.writeObject(entry.getKey());
+                writer.writeObject(entry.getValue());
+            }
+        }
+        else {
+            for (Map.Entry<K, V> entry : map.entrySet())
+                writeClo.write(writer, entry.getKey(), entry.getValue());
+        }
+    }
+
+    /**
+     * Read collection.
+     *
+     * @param reader Reader.
+     * @return List.
+     */
+    public static <T> List<T> readCollection(PortableRawReaderEx reader) {
+        return readCollection(reader, null);
+    }
+
+    /**
+     * Read collection.
+     *
+     * @param reader Reader.
+     * @param readClo Optional reader closure.
+     * @return List.
+     */
+    public static <T> List<T> readCollection(PortableRawReaderEx reader, @Nullable PlatformReaderClosure<T> readClo) {
+        int cnt = reader.readInt();
+
+        List<T> res = new ArrayList<>(cnt);
+
+        if (readClo == null) {
+            for (int i = 0; i < cnt; i++)
+                res.add((T)reader.readObjectDetached());
+        }
+        else {
+            for (int i = 0; i < cnt; i++)
+                res.add(readClo.read(reader));
+        }
+
+        return res;
+    }
+
+    /**
+     * Read nullable collection.
+     *
+     * @param reader Reader.
+     * @return List.
+     */
+    public static <T> List<T> readNullableCollection(PortableRawReaderEx reader) {
+        return readNullableCollection(reader, null);
+    }
+
+    /**
+     * Read nullable collection.
+     *
+     * @param reader Reader.
+     * @return List.
+     */
+    public static <T> List<T> readNullableCollection(PortableRawReaderEx reader,
+        @Nullable PlatformReaderClosure<T> readClo) {
+        if (!reader.readBoolean())
+            return null;
+
+        return readCollection(reader, readClo);
+    }
+
+    /**
+     * @param reader Reader.
+     * @return Set.
+     */
+    public static <T> Set<T> readSet(PortableRawReaderEx reader) {
+        int cnt = reader.readInt();
+
+        Set<T> res = U.newHashSet(cnt);
+
+        for (int i = 0; i < cnt; i++)
+            res.add((T)reader.readObjectDetached());
+
+        return res;
+    }
+
+    /**
+     * @param reader Reader.
+     * @return Set.
+     */
+    public static <T> Set<T> readNullableSet(PortableRawReaderEx reader) {
+        if (!reader.readBoolean())
+            return null;
+
+        return readSet(reader);
+    }
+
+    /**
+     * Read map.
+     *
+     * @param reader Reader.
+     * @return Map.
+     */
+    public static <K, V> Map<K, V> readMap(PortableRawReaderEx reader) {
+        return readMap(reader, null);
+    }
+
+    /**
+     * Read map.
+     *
+     * @param reader Reader.
+     * @param readClo Reader closure.
+     * @return Map.
+     */
+    public static <K, V> Map<K, V> readMap(PortableRawReaderEx reader,
+        @Nullable PlatformReaderBiClosure<K, V> readClo) {
+        int cnt = reader.readInt();
+
+        Map<K, V> map = U.newHashMap(cnt);
+
+        if (readClo == null) {
+            for (int i = 0; i < cnt; i++)
+                map.put((K)reader.readObjectDetached(), (V)reader.readObjectDetached());
+        }
+        else {
+            for (int i = 0; i < cnt; i++) {
+                IgniteBiTuple<K, V> entry = readClo.read(reader);
+
+                map.put(entry.getKey(), entry.getValue());
+            }
+        }
+
+        return map;
+    }
+
+    /**
+     * Read nullable map.
+     *
+     * @param reader Reader.
+     * @return Map.
+     */
+    public static <K, V> Map<K, V> readNullableMap(PortableRawReaderEx reader) {
+        if (!reader.readBoolean())
+            return null;
+
+        return readMap(reader);
+    }
+
+    /**
+     * Writes IgniteUuid to a writer.
+     *
+     * @param writer Writer.
+     * @param val Values.
+     */
+    public static void writeIgniteUuid(PortableRawWriterEx writer, IgniteUuid val) {
+        if (val == null)
+            writer.writeUuid(null);
+        else {
+            writer.writeUuid(val.globalId());
+            writer.writeLong(val.localId());
+        }
+    }
+
+    /**
+     * Convert native cache peek modes to Java cache peek modes.
+     *
+     * @param modes Encoded peek modes.
+     * @return Cache peek modes.
+     */
+    public static CachePeekMode[] decodeCachePeekModes(int modes) {
+        // 1. Try getting cache value.
+        CachePeekMode[] res = CACHE_PEEK_MODES[modes];
+
+        if (res == null) {
+            // 2. Calculate modes from scratch.
+            List<CachePeekMode> res0 = new ArrayList<>(CACHE_PEEK_MODES_CNT);
+
+            for (int i = 0; i < CACHE_PEEK_MODES_CNT; i++) {
+                int mask = 1 << i;
+
+                if ((modes & mask) == mask)
+                    res0.add(CachePeekMode.fromOrdinal((byte)i));
+            }
+
+            res = res0.toArray(new CachePeekMode[res0.size()]);
+
+            synchronized (PlatformUtils.class) {
+                CACHE_PEEK_MODES[modes] = res;
+            }
+        }
+
+        return res;
+    }
+
+    /**
+     * Unwrap query exception.
+     *
+     * @param err Initial error.
+     * @return Unwrapped error.
+     */
+    public static IgniteCheckedException unwrapQueryException(Throwable err) {
+        assert err != null;
+
+        Throwable parent = err;
+        Throwable child = parent.getCause();
+
+        while (true) {
+            if (child == null || child == parent)
+                break;
+
+            if (child instanceof IgniteException || child instanceof IgniteCheckedException
+                || child instanceof CacheException) {
+                // Continue unwrapping.
+                parent = child;
+
+                child = parent.getCause();
+
+                continue;
+            }
+
+            break;
+        }
+
+        // Specific exception found, but detailed message doesn't exist. Just pass exception name then.
+        if (parent.getMessage() == null)
+            return new IgniteCheckedException("Query execution failed due to exception: " +
+                parent.getClass().getName(), err);
+
+        return new IgniteCheckedException(parent.getMessage(), err);
+    }
+
+    /**
+     * Apply continuous query events to listener.
+     *
+     * @param ctx Context.
+     * @param lsnrPtr Listener pointer.
+     * @param evts Events.
+     * @throws javax.cache.event.CacheEntryListenerException In case of failure.
+     */
+    public static void applyContinuousQueryEvents(PlatformContext ctx, long lsnrPtr, Iterable<CacheEntryEvent> evts)
+        throws CacheEntryListenerException {
+        assert lsnrPtr != 0;
+        assert evts != null;
+
+        try (PlatformMemory mem = ctx.memory().allocate()) {
+            PlatformOutputStream out = mem.output();
+
+            PortableRawWriterEx writer = ctx.writer(out);
+
+            int cntPos = writer.reserveInt();
+
+            int cnt = 0;
+
+            for (CacheEntryEvent evt : evts) {
+                writeCacheEntryEvent(writer, evt);
+
+                cnt++;
+            }
+
+            writer.writeInt(cntPos, cnt);
+
+            out.synchronize();
+
+            ctx.gateway().continuousQueryListenerApply(lsnrPtr, mem.pointer());
+        }
+        catch (Exception e) {
+            throw toCacheEntryListenerException(e);
+        }
+    }
+
+    /**
+     * Evaluate the filter.
+     *
+     * @param ctx Context.
+     * @param filterPtr Native filter pointer.
+     * @param evt Event.
+     * @return Result.
+     * @throws CacheEntryListenerException In case of failure.
+     */
+    public static boolean evaluateContinuousQueryEvent(PlatformContext ctx, long filterPtr, CacheEntryEvent evt)
+        throws CacheEntryListenerException {
+        assert filterPtr != 0;
+
+        try (PlatformMemory mem = ctx.memory().allocate()) {
+            PlatformOutputStream out = mem.output();
+
+            writeCacheEntryEvent(ctx.writer(out), evt);
+
+            out.synchronize();
+
+            return ctx.gateway().continuousQueryFilterApply(filterPtr, mem.pointer()) == 1;
+        }
+        catch (Exception e) {
+            throw toCacheEntryListenerException(e);
+        }
+    }
+
+    /**
+     * Convert exception into listener exception.
+     *
+     * @param e Listener exception.
+     * @return Exception.
+     */
+    private static CacheEntryListenerException toCacheEntryListenerException(Exception e) {
+        assert e != null;
+
+        return e instanceof CacheEntryListenerException ? (CacheEntryListenerException)e : e.getMessage() != null ?
+            new CacheEntryListenerException(e.getMessage(), e) : new CacheEntryListenerException(e);
+    }
+
+    /**
+     * Write event to the writer.
+     *
+     * @param writer Writer.
+     * @param evt Event.
+     */
+    private static void writeCacheEntryEvent(PortableRawWriterEx writer, CacheEntryEvent evt) {
+        writer.writeObjectDetached(evt.getKey());
+        writer.writeObjectDetached(evt.getOldValue());
+        writer.writeObjectDetached(evt.getValue());
+    }
+
+    /**
+     * Writer error data.
+     *
+     * @param err Error.
+     * @param writer Writer.
+     */
+    public static void writeErrorData(Throwable err, PortableRawWriterEx writer) {
+        writeErrorData(err, writer, null);
+    }
+
+    /**
+     * Write error data.
+     * @param err Error.
+     * @param writer Writer.
+     * @param log Optional logger.
+     */
+    public static void writeErrorData(Throwable err, PortableRawWriterEx writer, @Nullable IgniteLogger log) {
+        // Write additional data if needed.
+        if (err instanceof PlatformExtendedException) {
+            PlatformExtendedException err0 = (PlatformExtendedException)err;
+
+            writer.writeBoolean(true); // Data exists.
+
+            int pos = writer.out().position();
+
+            try {
+                writer.writeBoolean(true); // Optimistically assume that we will be able to write it.
+                err0.writeData(writer);
+            }
+            catch (Exception e) {
+                if (log != null)
+                    U.warn(log, "Failed to write interop exception data: " + e.getMessage(), e);
+
+                writer.out().position(pos);
+
+                writer.writeBoolean(false); // Error occurred.
+                writer.writeString(e.getClass().getName());
+
+                String innerMsg;
+
+                try {
+                    innerMsg = e.getMessage();
+                }
+                catch (Exception innerErr) {
+                    innerMsg = "Exception message is not available.";
+                }
+
+                writer.writeString(innerMsg);
+            }
+        }
+        else
+            writer.writeBoolean(false);
+    }
+
+    /**
+     * Get GridGain platform processor.
+     *
+     * @param grid Ignite instance.
+     * @return Platform processor.
+     */
+    public static PlatformProcessor platformProcessor(Ignite grid) {
+        GridKernalContext ctx = ((IgniteKernal) grid).context();
+
+        return ctx.platform();
+    }
+
+    /**
+     * Gets interop context for the grid.
+     *
+     * @param grid Grid
+     * @return Context.
+     */
+    public static PlatformContext platformContext(Ignite grid) {
+        return platformProcessor(grid).context();
+    }
+
+    /**
+     * Reallocate arbitrary memory chunk.
+     *
+     * @param memPtr Memory pointer.
+     * @param cap Capacity.
+     */
+    public static void reallocate(long memPtr, int cap) {
+        PlatformMemoryUtils.reallocate(memPtr, cap);
+    }
+
+    /**
+     * Get error data.
+     *
+     * @param err Error.
+     * @return Error data.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public static byte[] errorData(Throwable err) {
+        if (err instanceof PlatformExtendedException) {
+            PlatformContext ctx = ((PlatformExtendedException)err).context();
+
+            try (PlatformMemory mem = ctx.memory().allocate()) {
+                // Write error data.
+                PlatformOutputStream out = mem.output();
+
+                PortableRawWriterEx writer = ctx.writer(out);
+
+                try {
+                    PlatformUtils.writeErrorData(err, writer, ctx.kernalContext().log(PlatformContext.class));
+                }
+                finally {
+                    out.synchronize();
+                }
+
+                // Read error data into separate array.
+                PlatformInputStream in = mem.input();
+
+                in.synchronize();
+
+                int len = in.remaining();
+
+                assert len > 0;
+
+                byte[] arr = in.array();
+                byte[] res = new byte[len];
+
+                System.arraycopy(arr, 0, res, 0, len);
+
+                return res;
+            }
+        }
+        else
+            return null;
+    }
+
+    /**
+     * Writes invocation result (of a job/service/etc) using a common protocol.
+     *
+     * @param writer Writer.
+     * @param resObj Result.
+     * @param err Error.
+     */
+    public static void writeInvocationResult(PortableRawWriterEx writer, Object resObj, Exception err)
+    {
+        if (err == null) {
+            writer.writeBoolean(true);
+            writer.writeObject(resObj);
+        }
+        else {
+            writer.writeBoolean(false);
+
+            PlatformNativeException nativeErr = null;
+
+            if (err instanceof IgniteCheckedException)
+                nativeErr = ((IgniteCheckedException)err).getCause(PlatformNativeException.class);
+            else if (err instanceof IgniteException)
+                nativeErr = ((IgniteException)err).getCause(PlatformNativeException.class);
+
+            if (nativeErr == null) {
+                writer.writeBoolean(false);
+                writer.writeString(err.getClass().getName());
+                writer.writeString(err.getMessage());
+            }
+            else {
+                writer.writeBoolean(true);
+                writer.writeObject(nativeErr.cause());
+            }
+        }
+    }
+
+    /**
+     * Reads invocation result (of a job/service/etc) using a common protocol.
+     *
+     * @param ctx Platform context.
+     * @param reader Reader.
+     * @return Result.
+     * @throws IgniteCheckedException When invocation result is an error.
+     */
+    public static Object readInvocationResult(PlatformContext ctx, PortableRawReaderEx reader)
+        throws IgniteCheckedException {
+        // 1. Read success flag.
+        boolean success = reader.readBoolean();
+
+        if (success)
+            // 2. Return result as is.
+            return reader.readObjectDetached();
+        else {
+            // 3. Read whether exception is in form of object or string.
+            boolean hasException = reader.readBoolean();
+
+            if (hasException) {
+                // 4. Full exception.
+                Object nativeErr = reader.readObjectDetached();
+
+                assert nativeErr != null;
+
+                throw ctx.createNativeException(nativeErr);
+            }
+            else {
+                // 5. Native exception was not serializable, we have only message.
+                String errMsg = reader.readString();
+
+                assert errMsg != null;
+
+                throw new IgniteCheckedException(errMsg);
+            }
+        }
+    }
+
+    /**
+     * Write .Net configuration to the stream.
+     *
+     * @param writer Writer.
+     * @param cfg Configuration.
+     */
+    public static void writeDotNetConfiguration(PortableRawWriterEx writer, PlatformDotNetConfiguration cfg) {
+        // 1. Write assemblies.
+        writeNullableCollection(writer, cfg.getAssemblies());
+
+        PlatformDotNetPortableConfiguration portableCfg = cfg.getPortableConfiguration();
+
+        if (portableCfg != null) {
+            writer.writeBoolean(true);
+
+            writeNullableCollection(writer, portableCfg.getTypesConfiguration(),
+                new PlatformWriterClosure<PlatformDotNetPortableTypeConfiguration>() {
+                @Override public void write(PortableRawWriterEx writer, PlatformDotNetPortableTypeConfiguration typ) {
+                    writer.writeString(typ.getAssemblyName());
+                    writer.writeString(typ.getTypeName());
+                    writer.writeString(typ.getNameMapper());
+                    writer.writeString(typ.getIdMapper());
+                    writer.writeString(typ.getSerializer());
+                    writer.writeString(typ.getAffinityKeyFieldName());
+                    writer.writeObject(typ.getMetadataEnabled());
+                    writer.writeObject(typ.getKeepDeserialized());
+                }
+            });
+
+            writeNullableCollection(writer, portableCfg.getTypes());
+            writer.writeString(portableCfg.getDefaultNameMapper());
+            writer.writeString(portableCfg.getDefaultIdMapper());
+            writer.writeString(portableCfg.getDefaultSerializer());
+            writer.writeBoolean(portableCfg.isDefaultMetadataEnabled());
+            writer.writeBoolean(portableCfg.isDefaultKeepDeserialized());
+        }
+        else
+            writer.writeBoolean(false);
+    }
+
+    /**
+     * Private constructor.
+     */
+    private PlatformUtils() {
+        // No-op.
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8045c820/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformWriterBiClosure.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformWriterBiClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformWriterBiClosure.java
new file mode 100644
index 0000000..6b04ad3
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformWriterBiClosure.java
@@ -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.
+ */
+
+package org.apache.ignite.internal.processors.platform.utils;
+
+import org.apache.ignite.internal.portable.PortableRawWriterEx;
+
+/**
+ * Interop writer bi-closure.
+ */
+public interface PlatformWriterBiClosure<T1, T2> {
+    /**
+     * Write values.
+     *
+     * @param writer Writer.
+     * @param val1 Value 1.
+     * @param val2 Value 2.
+     */
+    public void write(PortableRawWriterEx writer, T1 val1, T2 val2);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8045c820/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformWriterClosure.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformWriterClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformWriterClosure.java
new file mode 100644
index 0000000..a67d70a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformWriterClosure.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.platform.utils;
+
+import org.apache.ignite.internal.portable.PortableRawWriterEx;
+
+/**
+ * Interop writer closure.
+ */
+public interface PlatformWriterClosure<T> {
+    /**
+     * Write value.
+     *
+     * @param writer Writer.
+     * @param val Value.
+     */
+    public void write(PortableRawWriterEx writer, T val);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8045c820/modules/core/src/main/java/org/apache/ignite/platform/cpp/PlatformCppConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/platform/cpp/PlatformCppConfiguration.java b/modules/core/src/main/java/org/apache/ignite/platform/cpp/PlatformCppConfiguration.java
new file mode 100644
index 0000000..18f8a43
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/platform/cpp/PlatformCppConfiguration.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.cpp;
+
+import org.apache.ignite.configuration.PlatformConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Platform CPP configuration.
+ */
+public class PlatformCppConfiguration implements PlatformConfiguration {
+    /**
+     * Default constructor.
+     */
+    public PlatformCppConfiguration() {
+        // No-op.
+    }
+
+    /**
+     * Copying constructor.
+     *
+     * @param cfg Configuration to copy.
+     */
+    public PlatformCppConfiguration(PlatformConfiguration cfg) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(PlatformConfiguration.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8045c820/modules/core/src/main/java/org/apache/ignite/platform/cpp/package-info.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/platform/cpp/package-info.java b/modules/core/src/main/java/org/apache/ignite/platform/cpp/package-info.java
new file mode 100644
index 0000000..6d745a7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/platform/cpp/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 description. -->
+ * Contains C++ platform-related classes.
+ */
+package org.apache.ignite.platform.cpp;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8045c820/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetCacheStoreFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetCacheStoreFactory.java b/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetCacheStoreFactory.java
new file mode 100644
index 0000000..97f0dce
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetCacheStoreFactory.java
@@ -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.
+ */
+
+package org.apache.ignite.platform.dotnet;
+
+import org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore;
+
+import javax.cache.configuration.Factory;
+import java.util.Map;
+
+/**
+ * Wrapper for .NET cache store implementations.
+ * <p>
+ * This wrapper should be used if you have an implementation of
+ * {@code GridGain.Cache.IGridCacheStore} interface in .NET and
+ * would like to configure it a persistence storage for your cache.
+ * To do tis you will need to configure the wrapper via
+ * {@link org.apache.ignite.configuration.CacheConfiguration#setCacheStoreFactory(javax.cache.configuration.Factory)} property
+ * and provide assembly name and class name of your .NET store
+ * implementation (both properties are mandatory):
+ * <pre name="code" class="xml">
+ * &lt;bean class="org.apache.ignite.cache.CacheConfiguration"&gt;
+ *     ...
+ *     &lt;property name="cacheStoreFactory"&gt;
+ *         &lt;bean class="org.gridgain.grid.interop.dotnet.InteropDotNetCacheStoreFactory"&gt;
+ *             &lt;property name="assemblyName" value="MyAssembly"/&gt;
+ *             &lt;property name="className" value="MyApp.MyCacheStore"/&gt;
+ *         &lt;/bean&gt;
+ *     &lt;/property&gt;
+ *     ...
+ * &lt;/bean&gt;
+ * </pre>
+ * If properly configured, this wrapper will instantiate an instance
+ * of your cache store in .NET and delegate all calls to that instance.
+ * To create an instance, assembly name and class name are passed to
+ * <a target="_blank" href="http://msdn.microsoft.com/en-us/library/d133hta4.aspx">System.Activator.CreateInstance(String, String)</a>
+ * method in .NET during node startup. Refer to its documentation for
+ * details.
+ */
+public class PlatformDotNetCacheStoreFactory implements Factory<PlatformDotNetCacheStore> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** .Net assembly name. */
+    private String assemblyName;
+
+    /** .Net class name. */
+    private String clsName;
+
+    /** Properties. */
+    private Map<String, ?> props;
+
+    /** Instance. */
+    private transient PlatformDotNetCacheStore instance;
+
+    /**
+     * Gets .NET assembly name.
+     *
+     * @return .NET assembly name.
+     */
+    public String getAssemblyName() {
+        return assemblyName;
+    }
+
+    /**
+     * Set .NET assembly name.
+     *
+     * @param assemblyName .NET assembly name.
+     */
+    public void setAssemblyName(String assemblyName) {
+        this.assemblyName = assemblyName;
+    }
+
+    /**
+     * Gets .NET class name.
+     *
+     * @return .NET class name.
+     */
+    public String getClassName() {
+        return clsName;
+    }
+
+    /**
+     * Sets .NET class name.
+     *
+     * @param clsName .NET class name.
+     */
+    public void setClassName(String clsName) {
+        this.clsName = clsName;
+    }
+
+    /**
+     * Get properties.
+     *
+     * @return Properties.
+     */
+    public Map<String, ?> getProperties() {
+        return props;
+    }
+
+    /**
+     * Set properties.
+     *
+     * @param props Properties.
+     */
+    public void setProperties(Map<String, ?> props) {
+        this.props = props;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public PlatformDotNetCacheStore create() {
+        synchronized (this) {
+            if (instance == null) {
+                instance = new PlatformDotNetCacheStore();
+
+                instance.setAssemblyName(assemblyName);
+                instance.setClassName(clsName);
+                instance.setProperties(props);
+            }
+
+            return instance;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8045c820/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..0550bab
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetConfiguration.java
@@ -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.
+ */
+
+package org.apache.ignite.platform.dotnet;
+
+import org.apache.ignite.configuration.PlatformConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Mirror of .Net class Configuration.cs
+ */
+public class PlatformDotNetConfiguration implements PlatformConfiguration {
+    /** */
+    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 String toString() {
+        return S.toString(PlatformDotNetConfiguration.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8045c820/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetLifecycleBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetLifecycleBean.java b/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetLifecycleBean.java
new file mode 100644
index 0000000..8e4b590
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetLifecycleBean.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.lifecycle.PlatformLifecycleBean;
+import org.apache.ignite.lifecycle.LifecycleBean;
+
+import java.util.Map;
+
+/**
+ * Lifecycle bean implementation which can be used to configure .Net lifecycle beans in Java Spring configuration.
+ */
+public class PlatformDotNetLifecycleBean extends PlatformLifecycleBean implements LifecycleBean {
+    /** Assembly name. */
+    private String assemblyName;
+
+    /** Class name. */
+    private String clsName;
+
+    /** Properties. */
+    private Map<String, ?> props;
+
+    /**
+     * Constructor.
+     */
+    public PlatformDotNetLifecycleBean() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param assemblyName Assembly name.
+     * @param clsName Class name.
+     */
+    public PlatformDotNetLifecycleBean(String assemblyName, String clsName) {
+        this.assemblyName = assemblyName;
+        this.clsName = clsName;
+    }
+
+    /**
+     * Get assembly name.
+     *
+     * @return Assembly name.
+     */
+    public String getAssemblyName() {
+        return assemblyName;
+    }
+
+    /**
+     * Set assembly name.
+     *
+     * @param assemblyName Assembly name.
+     */
+    public void setAssemblyName(String assemblyName) {
+        this.assemblyName = assemblyName;
+    }
+
+    /**
+     * Get class name.
+     *
+     * @return Class name.
+     */
+    public String getClassName() {
+        return clsName;
+    }
+
+    /**
+     * Set class name.
+     *
+     * @param clsName Class name.
+     */
+    public void setClassName(String clsName) {
+        this.clsName = clsName;
+    }
+
+    /**
+     * Get properties.
+     *
+     * @return Properties.
+     */
+    public Map<String, ?> getProperties() {
+        return props;
+    }
+
+    /**
+     * Set properties.
+     *
+     * @param props Properties.
+     */
+    public void setProperties(Map<String, ?> props) {
+        this.props = props;
+    }
+}
\ No newline at end of file