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/03/09 10:27:46 UTC

[31/50] [abbrv] ignite git commit: IGNITE-2621: Correct handling for tasks in mixed-platform cluster. This closes #505.

IGNITE-2621: Correct handling for tasks in mixed-platform cluster. This closes #505.


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

Branch: refs/heads/ignite-1786
Commit: 2d38eb8fe2567bba86a736a6cec7bb85808d1d60
Parents: ee01b61
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Thu Mar 3 09:52:13 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Mar 3 09:52:13 2016 +0300

----------------------------------------------------------------------
 .../processors/platform/PlatformContext.java    |   9 +-
 .../platform/PlatformContextImpl.java           |  13 +-
 .../platform/PlatformProcessorImpl.java         |  14 +--
 .../platform/compute/PlatformCompute.java       |  23 +++-
 .../platform/PlatformStartIgniteTask.java       |  77 ++++++++++++
 .../ignite/platform/PlatformStopIgniteTask.java |  74 +++++++++++
 .../Apache.Ignite.Core.Tests.csproj             |   1 +
 .../Compute/MixedClusterTest.cs                 | 123 +++++++++++++++++++
 .../Config/Compute/compute-grid1.xml            |   1 -
 .../Config/Compute/compute-grid2.xml            |  15 ++-
 .../IgniteConfigurationTest.cs                  |  17 +--
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |  33 ++++-
 12 files changed, 365 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2d38eb8f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
index b05d331..e88d57b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
@@ -272,4 +272,11 @@ public interface PlatformContext {
      * @return Cluster node filter.
      */
     public PlatformClusterNodeFilter createClusterNodeFilter(Object filter);
-}
+
+    /**
+     * Gets the current platform name.
+     *
+     * @return Current platform name.
+     */
+    public String platform();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/2d38eb8f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
index c531718..b45414a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
@@ -108,6 +108,9 @@ public class PlatformContextImpl implements PlatformContext {
     /** Node ids that has been sent to native platform. */
     private final Set<UUID> sentNodes = Collections.newSetFromMap(new ConcurrentHashMap<UUID, Boolean>());
 
+    /** Platform name. */
+    private final String platform;
+
     /**
      * Static initializer.
      */
@@ -142,11 +145,14 @@ public class PlatformContextImpl implements PlatformContext {
      * @param ctx Kernal context.
      * @param gate Callback gateway.
      * @param mem Memory manager.
+     * @param platform Platform name.
      */
-    public PlatformContextImpl(GridKernalContext ctx, PlatformCallbackGateway gate, PlatformMemoryManagerImpl mem) {
+    public PlatformContextImpl(GridKernalContext ctx, PlatformCallbackGateway gate, PlatformMemoryManagerImpl mem,
+        String platform) {
         this.ctx = ctx;
         this.gate = gate;
         this.mem = mem;
+        this.platform = platform;
 
         cacheObjProc = (CacheObjectBinaryProcessorImpl)ctx.cacheObjects();
 
@@ -621,6 +627,11 @@ public class PlatformContextImpl implements PlatformContext {
         return new PlatformClusterNodeFilterImpl(filter, this);
     }
 
+    /** {@inheritDoc} */
+    @Override public String platform() {
+        return platform;
+    }
+
     /**
      * Metadata holder.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/2d38eb8f/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 76967ff..95daa4d 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
@@ -26,11 +26,9 @@ 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.IgniteInternalFuture;
 import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.binary.BinaryRawWriterEx;
-import org.apache.ignite.internal.cluster.ClusterGroupAdapter;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl;
@@ -122,7 +120,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
                 U.warn(log, w);
         }
 
-        platformCtx = new PlatformContextImpl(ctx, interopCfg.gate(), interopCfg.memory());
+        platformCtx = new PlatformContextImpl(ctx, interopCfg.gate(), interopCfg.memory(), interopCfg.platform());
     }
 
     /** {@inheritDoc} */
@@ -209,12 +207,12 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
     }
 
     /** {@inheritDoc} */
-    public void releaseStart() {
+    @Override public void releaseStart() {
         startLatch.countDown();
     }
 
     /** {@inheritDoc} */
-    public void awaitStart() throws IgniteCheckedException {
+    @Override public void awaitStart() throws IgniteCheckedException {
         U.await(startLatch);
     }
 
@@ -305,9 +303,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
     @Override public PlatformTarget compute(PlatformTarget grp) {
         PlatformClusterGroup grp0 = (PlatformClusterGroup)grp;
 
-        assert grp0.projection() instanceof ClusterGroupAdapter; // Safety for very complex ClusterGroup hierarchy.
-
-        return new PlatformCompute(platformCtx, (IgniteComputeImpl)((ClusterGroupAdapter)grp0.projection()).compute());
+        return new PlatformCompute(platformCtx, grp0.projection());
     }
 
     /** {@inheritDoc} */
@@ -343,7 +339,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
 
         try {
             if (stopped)
-                throw new IgniteCheckedException("Failed to initialize interop store becuase node is stopping: " +
+                throw new IgniteCheckedException("Failed to initialize interop store because node is stopping: " +
                     store);
 
             if (started)

http://git-wip-us.apache.org/repos/asf/ignite/blob/2d38eb8f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
index 10545d5..a1a82ae 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.platform.compute;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteCompute;
 import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.compute.ComputeTaskFuture;
 import org.apache.ignite.internal.IgniteComputeImpl;
 import org.apache.ignite.internal.IgniteInternalFuture;
@@ -30,6 +31,7 @@ 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.processors.platform.utils.PlatformListenable;
+import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
 import org.apache.ignite.internal.util.future.IgniteFutureImpl;
 import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteInClosure;
@@ -65,18 +67,25 @@ public class PlatformCompute extends PlatformAbstractTarget {
     /** Compute instance. */
     private final IgniteComputeImpl compute;
 
+    /** Compute instance for platform-only nodes. */
+    private final IgniteComputeImpl computeForPlatform;
+
     /** Future for previous asynchronous operation. */
     protected ThreadLocal<IgniteInternalFuture> curFut = new ThreadLocal<>();
     /**
      * Constructor.
      *
      * @param platformCtx Context.
-     * @param compute Compute instance.
+     * @param grp Cluster group.
      */
-    public PlatformCompute(PlatformContext platformCtx, IgniteComputeImpl compute) {
+    public PlatformCompute(PlatformContext platformCtx, ClusterGroup grp) {
         super(platformCtx);
 
-        this.compute = compute;
+        compute = (IgniteComputeImpl)grp.ignite().compute(grp);
+
+        ClusterGroup platformGrp = grp.forAttribute(PlatformUtils.ATTR_PLATFORM, platformCtx.platform());
+
+        computeForPlatform = (IgniteComputeImpl)grp.ignite().compute(platformGrp);
     }
 
     /** {@inheritDoc} */
@@ -153,7 +162,7 @@ public class PlatformCompute extends PlatformAbstractTarget {
                 ((PlatformBalancingMultiClosureTask)task).jobs(jobs);
         }
 
-        platformCtx.kernalContext().task().setThreadContext(TC_SUBGRID, compute.clusterGroup().nodes());
+        platformCtx.kernalContext().task().setThreadContext(TC_SUBGRID, computeForPlatform.clusterGroup().nodes());
 
         return executeNative0(task);
     }
@@ -195,7 +204,7 @@ public class PlatformCompute extends PlatformAbstractTarget {
      * @param topVer Topology version.
      */
     public PlatformListenable executeNative(long taskPtr, long topVer) {
-        final PlatformFullTask task = new PlatformFullTask(platformCtx, compute, taskPtr, topVer);
+        final PlatformFullTask task = new PlatformFullTask(platformCtx, computeForPlatform, taskPtr, topVer);
 
         return executeNative0(task);
     }
@@ -207,6 +216,7 @@ public class PlatformCompute extends PlatformAbstractTarget {
      */
     public void withTimeout(long timeout) {
         compute.withTimeout(timeout);
+        computeForPlatform.withTimeout(timeout);
     }
 
     /**
@@ -214,6 +224,7 @@ public class PlatformCompute extends PlatformAbstractTarget {
      */
     public void withNoFailover() {
         compute.withNoFailover();
+        computeForPlatform.withNoFailover();
     }
 
     /** <inheritDoc /> */
@@ -232,7 +243,7 @@ public class PlatformCompute extends PlatformAbstractTarget {
      * @param task Task.
      */
     private PlatformListenable executeNative0(final PlatformAbstractTask task) {
-        IgniteInternalFuture fut = compute.executeAsync(task, null);
+        IgniteInternalFuture fut = computeForPlatform.executeAsync(task, null);
 
         fut.listen(new IgniteInClosure<IgniteInternalFuture>() {
             private static final long serialVersionUID = 0L;

http://git-wip-us.apache.org/repos/asf/ignite/blob/2d38eb8f/modules/core/src/test/java/org/apache/ignite/platform/PlatformStartIgniteTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/platform/PlatformStartIgniteTask.java b/modules/core/src/test/java/org/apache/ignite/platform/PlatformStartIgniteTask.java
new file mode 100644
index 0000000..c8eb13f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/platform/PlatformStartIgniteTask.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.platform;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.compute.ComputeJob;
+import org.apache.ignite.compute.ComputeJobAdapter;
+import org.apache.ignite.compute.ComputeJobResult;
+import org.apache.ignite.compute.ComputeTaskAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.jetbrains.annotations.Nullable;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Task to start an Ignite node.
+ */
+public class PlatformStartIgniteTask extends ComputeTaskAdapter<String, String> {
+    /** {@inheritDoc} */
+    @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid,
+        @Nullable String arg) throws IgniteException {
+        return Collections.singletonMap(new PlatformStartIgniteJob(arg), F.first(subgrid));
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public String reduce(List<ComputeJobResult> results) throws IgniteException {
+        ComputeJobResult res = results.get(0);
+
+        if (res.getException() != null)
+            throw res.getException();
+        else
+            return results.get(0).getData();
+    }
+
+    /**
+     * Job.
+     */
+    private static class PlatformStartIgniteJob extends ComputeJobAdapter {
+        /** */
+        private final String springConfig;
+
+        /**
+         * Ctor.
+         * @param springConfig Config.
+         */
+        private PlatformStartIgniteJob(String springConfig) {
+            this.springConfig = springConfig;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object execute() throws IgniteException {
+            Ignite ignite = Ignition.start(springConfig);
+
+            return ignite.name();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/2d38eb8f/modules/core/src/test/java/org/apache/ignite/platform/PlatformStopIgniteTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/platform/PlatformStopIgniteTask.java b/modules/core/src/test/java/org/apache/ignite/platform/PlatformStopIgniteTask.java
new file mode 100644
index 0000000..c0319ab
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/platform/PlatformStopIgniteTask.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.platform;
+
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.compute.ComputeJob;
+import org.apache.ignite.compute.ComputeJobAdapter;
+import org.apache.ignite.compute.ComputeJobResult;
+import org.apache.ignite.compute.ComputeTaskAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.jetbrains.annotations.Nullable;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Task to stop an Ignite node.
+ */
+public class PlatformStopIgniteTask extends ComputeTaskAdapter<String, Boolean> {
+    /** {@inheritDoc} */
+    @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid,
+        @Nullable String arg) throws IgniteException {
+        return Collections.singletonMap(new PlatformStopIgniteJob(arg), F.first(subgrid));
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Boolean reduce(List<ComputeJobResult> results) throws IgniteException {
+        ComputeJobResult res = results.get(0);
+
+        if (res.getException() != null)
+            throw res.getException();
+        else
+            return results.get(0).getData();
+    }
+
+    /**
+     * Job.
+     */
+    private static class PlatformStopIgniteJob extends ComputeJobAdapter {
+        /** */
+        private final String gridName;
+
+        /**
+         * Ctor.
+         * @param gridName Name.
+         */
+        private PlatformStopIgniteJob(String gridName) {
+            this.gridName = gridName;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object execute() throws IgniteException {
+            return Ignition.stop(gridName, true);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/2d38eb8f/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 4ba05e1..e72fb85 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
@@ -102,6 +102,7 @@
     <Compile Include="Compute\FailoverTaskSelfTest.cs" />
     <Compile Include="Compute\BinarizableClosureTaskTest.cs" />
     <Compile Include="Compute\BinarizableTaskTest.cs" />
+    <Compile Include="Compute\MixedClusterTest.cs" />
     <Compile Include="Compute\ResourceTaskTest.cs" />
     <Compile Include="Compute\SerializableClosureTaskTest.cs" />
     <Compile Include="Compute\TaskAdapterTest.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/2d38eb8f/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/MixedClusterTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/MixedClusterTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/MixedClusterTest.cs
new file mode 100644
index 0000000..57ea892
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/MixedClusterTest.cs
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma warning disable 618  // SpringConfigUrl
+namespace Apache.Ignite.Core.Tests.Compute
+{
+    using System;
+    using System.Collections;
+    using System.Linq;
+    using Apache.Ignite.Core.Compute;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests compute in a cluster with Java-only and .NET nodes.
+    /// </summary>
+    public class MixedClusterTest
+    {
+        /** */
+        private const string SpringConfig = @"Config\Compute\compute-grid1.xml";
+
+        /** */
+        private const string SpringConfig2 = @"Config\Compute\compute-grid2.xml";
+
+        /** */
+        private const string StartTask = "org.apache.ignite.platform.PlatformStartIgniteTask";
+
+        /** */
+        private const string StopTask = "org.apache.ignite.platform.PlatformStopIgniteTask";
+
+        /// <summary>
+        /// Tests the compute.
+        /// </summary>
+        [Test]
+        public void TestCompute()
+        {
+            var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration()) {SpringConfigUrl = SpringConfig};
+
+            using (var ignite = Ignition.Start(cfg))
+            {
+                var javaNodeName = StartJavaNode(ignite, SpringConfig2);
+
+                try
+                {
+                    Assert.IsTrue(ignite.WaitTopology(2));
+
+                    TestDotNetTask(ignite);
+                    TestJavaTask(ignite);
+                }
+                finally
+                {
+                    StopJavaNode(ignite, javaNodeName);
+                }
+            }
+        }
+
+        /// <summary>
+        /// Tests the dot net task.
+        /// </summary>
+        /// <param name="ignite">The ignite.</param>
+        private static void TestDotNetTask(IIgnite ignite)
+        {
+            var results = ignite.GetCompute().Broadcast(new ComputeFunc());
+
+            // There are two nodes, but only one can execute .NET jobs.
+            Assert.AreEqual(new[] {int.MaxValue}, results.ToArray());
+        }
+
+        /// <summary>
+        /// Tests the dot net task.
+        /// </summary>
+        /// <param name="ignite">The ignite.</param>
+        private static void TestJavaTask(IIgnite ignite)
+        {
+            // Java task can execute on both nodes.
+            var res = ignite.GetCompute().ExecuteJavaTask<ICollection>(ComputeApiTest.BroadcastTask, null);
+
+            Assert.AreEqual(2, res.Count);
+        }
+
+        /// <summary>
+        /// Starts the java node.
+        /// </summary>
+        private static string StartJavaNode(IIgnite grid, string config)
+        {
+            return grid.GetCompute().ExecuteJavaTask<string>(StartTask, config);
+        }
+
+        /// <summary>
+        /// Stops the java node.
+        /// </summary>
+        private static void StopJavaNode(IIgnite grid, string name)
+        {
+            grid.GetCompute().ExecuteJavaTask<object>(StopTask, name);
+        }
+
+        /// <summary>
+        /// Test func.
+        /// </summary>
+        [Serializable]
+        private class ComputeFunc : IComputeFunc<int>
+        {
+            /** <inheritdoc /> */
+            public int Invoke()
+            {
+                return int.MaxValue;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/2d38eb8f/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
index 3061773..78a30a8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
@@ -69,7 +69,6 @@
                         </bean>
                     </list>
                 </property>
-
             </bean>
         </property>
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/2d38eb8f/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid2.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid2.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid2.xml
index ef29a89..b1e8235 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid2.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid2.xml
@@ -62,10 +62,23 @@
                         </bean>
                     </list>
                 </property>
-
+                <property name="idMapper">
+                    <bean class="org.apache.ignite.binary.BinaryBasicIdMapper">
+                        <constructor-arg value="true"/>
+                    </bean>
+                </property>
+                <property name="nameMapper">
+                    <bean class="org.apache.ignite.binary.BinaryBasicNameMapper">
+                        <constructor-arg value="true"/>
+                    </bean>
+                </property>
             </bean>
         </property>
 
+        <property name="marshaller">
+            <bean class="org.apache.ignite.internal.binary.BinaryMarshaller" />
+        </property>
+
         <property name="discoverySpi">
             <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
                 <property name="ipFinder">

http://git-wip-us.apache.org/repos/asf/ignite/blob/2d38eb8f/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
index 15f5804..3aa26d8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
@@ -24,7 +24,6 @@ namespace Apache.Ignite.Core.Tests
     using System.Linq;
     using Apache.Ignite.Core.Cache.Configuration;
     using Apache.Ignite.Core.Common;
-    using Apache.Ignite.Core.Discovery;
     using Apache.Ignite.Core.Discovery.Tcp;
     using Apache.Ignite.Core.Discovery.Tcp.Multicast;
     using Apache.Ignite.Core.Discovery.Tcp.Static;
@@ -138,12 +137,12 @@ namespace Apache.Ignite.Core.Tests
             using (var ignite = Ignition.Start(new IgniteConfiguration
             {
                 Localhost = "127.0.0.1",
-                DiscoverySpi = GetStaticDiscovery()
+                DiscoverySpi = TestUtils.GetStaticDiscovery()
             }))
             using (var ignite2 = Ignition.Start(new IgniteConfiguration
             {
                 Localhost = "127.0.0.1",
-                DiscoverySpi = GetStaticDiscovery(),
+                DiscoverySpi = TestUtils.GetStaticDiscovery(),
                 GridName = "client",
                 ClientMode = true
             }))
@@ -351,17 +350,5 @@ namespace Apache.Ignite.Core.Tests
                 Localhost = "127.0.0.1"
             };
         }
-
-        /// <summary>
-        /// Gets the static discovery.
-        /// </summary>
-        /// <returns></returns>
-        private static IDiscoverySpi GetStaticDiscovery()
-        {
-            return new TcpDiscoverySpi
-            {
-                IpFinder = new TcpDiscoveryStaticIpFinder {Endpoints = new[] {"127.0.0.1:47500", "127.0.0.1:47501"}}
-            };
-        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/2d38eb8f/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
index f972cf7..9fe8f1c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
@@ -22,6 +22,9 @@ namespace Apache.Ignite.Core.Tests
     using System.Collections.Generic;
     using System.Linq;
     using System.Threading;
+    using Apache.Ignite.Core.Discovery;
+    using Apache.Ignite.Core.Discovery.Tcp;
+    using Apache.Ignite.Core.Discovery.Tcp.Static;
     using Apache.Ignite.Core.Impl;
     using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Tests.Process;
@@ -212,7 +215,7 @@ namespace Apache.Ignite.Core.Tests
         /// <returns>
         ///   <c>True</c> if topology took required size.
         /// </returns>
-        public static bool WaitTopology(this IIgnite grid, int size, int timeout)
+        public static bool WaitTopology(this IIgnite grid, int size, int timeout = 30000)
         {
             int left = timeout;
 
@@ -302,5 +305,33 @@ namespace Apache.Ignite.Core.Tests
 
             return false;
         }
+
+        /// <summary>
+        /// Gets the static discovery.
+        /// </summary>
+        public static IDiscoverySpi GetStaticDiscovery()
+        {
+            return new TcpDiscoverySpi
+            {
+                IpFinder = new TcpDiscoveryStaticIpFinder
+                {
+                    Endpoints = new[] { "127.0.0.1:47500", "127.0.0.1:47501" }
+                }
+            };
+        }
+
+        /// <summary>
+        /// Gets the default code-based test configuration.
+        /// </summary>
+        public static IgniteConfiguration GetTestConfiguration()
+        {
+            return new IgniteConfiguration
+            {
+                DiscoverySpi = GetStaticDiscovery(),
+                Localhost = "127.0.0.1",
+                JvmOptions = TestJavaOptions(),
+                JvmClasspath = CreateTestClasspath()
+            };
+        }
     }
 }