You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2017/04/06 10:45:51 UTC

[14/50] [abbrv] ignite git commit: IGNITE-4802 - Separate thread pool for managed services

IGNITE-4802 - Separate thread pool for managed services


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

Branch: refs/heads/ignite-3477-master
Commit: 1f3b2fcd003c1f084874d5c421953da0a7cd02cb
Parents: 3aa2a68
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Mon Mar 27 18:12:17 2017 -0700
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Mon Mar 27 18:12:17 2017 -0700

----------------------------------------------------------------------
 .../configuration/IgniteConfiguration.java      |  30 +++++
 .../ignite/internal/GridKernalContext.java      |   7 +
 .../ignite/internal/GridKernalContextImpl.java  |  11 ++
 .../apache/ignite/internal/IgniteKernal.java    |   2 +
 .../org/apache/ignite/internal/IgnitionEx.java  |  16 +++
 .../managers/communication/GridIoManager.java   |   2 +
 .../managers/communication/GridIoPolicy.java    |   5 +-
 .../internal/processors/pool/PoolProcessor.java |  10 +-
 .../processors/service/GridServiceProxy.java    |  10 ++
 .../task/GridTaskThreadContextKey.java          |   7 +-
 .../processors/task/GridTaskWorker.java         |  16 ++-
 .../services/ServiceThreadPoolSelfTest.java     | 133 +++++++++++++++++++
 .../junits/GridTestKernalContext.java           |   1 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |   2 +
 14 files changed, 245 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1f3b2fcd/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index dcd8a80..b240446 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -53,6 +53,7 @@ import org.apache.ignite.plugin.PluginConfiguration;
 import org.apache.ignite.plugin.PluginProvider;
 import org.apache.ignite.plugin.segmentation.SegmentationPolicy;
 import org.apache.ignite.plugin.segmentation.SegmentationResolver;
+import org.apache.ignite.services.Service;
 import org.apache.ignite.services.ServiceConfiguration;
 import org.apache.ignite.spi.checkpoint.CheckpointSpi;
 import org.apache.ignite.spi.checkpoint.noop.NoopCheckpointSpi;
@@ -233,6 +234,9 @@ public class IgniteConfiguration {
     /** Public pool size. */
     private int pubPoolSize = DFLT_PUBLIC_THREAD_CNT;
 
+    /** Service pool size. */
+    private Integer svcPoolSize;
+
     /** Async Callback pool size. */
     private int callbackPoolSize = DFLT_PUBLIC_THREAD_CNT;
 
@@ -561,6 +565,7 @@ public class IgniteConfiguration {
         storeSesLsnrs = cfg.getCacheStoreSessionListenerFactories();
         stripedPoolSize = cfg.getStripedPoolSize();
         svcCfgs = cfg.getServiceConfiguration();
+        svcPoolSize = cfg.getServiceThreadPoolSize();
         sysPoolSize = cfg.getSystemThreadPoolSize();
         timeSrvPortBase = cfg.getTimeServerPortBase();
         timeSrvPortRange = cfg.getTimeServerPortRange();
@@ -774,6 +779,18 @@ public class IgniteConfiguration {
     }
 
     /**
+     * Should return a thread pool size to be used in grid.
+     * This executor service will be in charge of processing {@link Service} proxy invocations.
+     * <p>
+     * If not provided, executor service will have size {@link #DFLT_PUBLIC_THREAD_CNT}.
+     *
+     * @return Thread pool size to be used in grid to process service proxy invocations.
+     */
+    public int getServiceThreadPoolSize() {
+        return svcPoolSize != null ? svcPoolSize : getPublicThreadPoolSize();
+    }
+
+    /**
      * Size of thread pool that is in charge of processing internal system messages.
      * <p>
      * If not provided, executor service will have size {@link #DFLT_SYSTEM_CORE_THREAD_CNT}.
@@ -894,6 +911,19 @@ public class IgniteConfiguration {
     }
 
     /**
+     * Sets thread pool size to use within grid.
+     *
+     * @param poolSize Thread pool size to use within grid.
+     * @see IgniteConfiguration#getServiceThreadPoolSize()
+     * @return {@code this} for chaining.
+     */
+    public IgniteConfiguration setServiceThreadPoolSize(int poolSize) {
+        svcPoolSize = poolSize;
+
+        return this;
+    }
+
+    /**
      * Sets system thread pool size to use within grid.
      *
      * @param poolSize Thread pool size to use within grid.

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f3b2fcd/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
index 927944f..d95f09c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
@@ -505,6 +505,13 @@ public interface GridKernalContext extends Iterable<GridComponent> {
     public ExecutorService getExecutorService();
 
     /**
+     * Executor service that is in charge of processing service proxy invocations.
+     *
+     * @return Thread pool implementation to be used in grid for service proxy invocations.
+     */
+    public ExecutorService getServiceExecutorService();
+
+    /**
      * Executor service that is in charge of processing internal system messages.
      *
      * @return Thread pool implementation to be used in grid for internal system messages.

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f3b2fcd/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
index a2ad1b2..8e1fd7e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
@@ -297,6 +297,10 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
 
     /** */
     @GridToStringExclude
+    protected ExecutorService svcExecSvc;
+
+    /** */
+    @GridToStringExclude
     protected ExecutorService sysExecSvc;
 
     /** */
@@ -405,6 +409,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
         ExecutorService utilityCachePool,
         ExecutorService marshCachePool,
         ExecutorService execSvc,
+        ExecutorService svcExecSvc,
         ExecutorService sysExecSvc,
         StripedExecutor stripedExecSvc,
         ExecutorService p2pExecSvc,
@@ -426,6 +431,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
         this.utilityCachePool = utilityCachePool;
         this.marshCachePool = marshCachePool;
         this.execSvc = execSvc;
+        this.svcExecSvc = svcExecSvc;
         this.sysExecSvc = sysExecSvc;
         this.stripedExecSvc = stripedExecSvc;
         this.p2pExecSvc = p2pExecSvc;
@@ -952,6 +958,11 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     }
 
     /** {@inheritDoc} */
+    @Override public ExecutorService getServiceExecutorService() {
+        return svcExecSvc;
+    }
+
+    /** {@inheritDoc} */
     @Override public ExecutorService getSystemExecutorService() {
         return sysExecSvc;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f3b2fcd/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 063fe25..7161166 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -686,6 +686,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         ExecutorService utilityCachePool,
         ExecutorService marshCachePool,
         final ExecutorService execSvc,
+        final ExecutorService svcExecSvc,
         final ExecutorService sysExecSvc,
         final StripedExecutor stripedExecSvc,
         ExecutorService p2pExecSvc,
@@ -795,6 +796,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                 utilityCachePool,
                 marshCachePool,
                 execSvc,
+                svcExecSvc,
                 sysExecSvc,
                 stripedExecSvc,
                 p2pExecSvc,

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f3b2fcd/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index f32a753..8212dd0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -1457,6 +1457,9 @@ public class IgnitionEx {
         /** Executor service. */
         private ThreadPoolExecutor execSvc;
 
+        /** Executor service for services. */
+        private ThreadPoolExecutor svcExecSvc;
+
         /** System executor service. */
         private ThreadPoolExecutor sysExecSvc;
 
@@ -1656,6 +1659,18 @@ public class IgnitionEx {
 
             execSvc.allowCoreThreadTimeOut(true);
 
+            validateThreadPoolSize(cfg.getServiceThreadPoolSize(), "service");
+
+            svcExecSvc = new IgniteThreadPoolExecutor(
+                "svc",
+                cfg.getGridName(),
+                cfg.getServiceThreadPoolSize(),
+                cfg.getServiceThreadPoolSize(),
+                DFLT_THREAD_KEEP_ALIVE_TIME,
+                new LinkedBlockingQueue<Runnable>());
+
+            svcExecSvc.allowCoreThreadTimeOut(true);
+
             validateThreadPoolSize(cfg.getSystemThreadPoolSize(), "system");
 
             sysExecSvc = new IgniteThreadPoolExecutor(
@@ -1801,6 +1816,7 @@ public class IgnitionEx {
                     utilityCacheExecSvc,
                     marshCacheExecSvc,
                     execSvc,
+                    svcExecSvc,
                     sysExecSvc,
                     stripedExecSvc,
                     p2pExecSvc,

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f3b2fcd/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
index 7ef7bc0..2eda4b7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
@@ -96,6 +96,7 @@ import static org.apache.ignite.internal.managers.communication.GridIoPolicy.MAN
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.MARSH_CACHE_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.P2P_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.PUBLIC_POOL;
+import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SERVICE_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.UTILITY_CACHE_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.isReservedGridIoPolicy;
@@ -686,6 +687,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
                 case MARSH_CACHE_POOL:
                 case IDX_POOL:
                 case IGFS_POOL:
+                case SERVICE_POOL:
                 {
                     if (msg.isOrdered())
                         processOrderedMessage(nodeId, msg, plc, msgC);

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f3b2fcd/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java
index 70a7354..bb64f6a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java
@@ -49,6 +49,9 @@ public class GridIoPolicy {
     /** Pool for handling distributed index range requests. */
     public static final byte IDX_POOL = 8;
 
+    /** Pool for service proxy executions. */
+    public static final byte SERVICE_POOL = 9;
+
     /**
      * Defines the range of reserved pools that are not available for plugins.
      * @param key The key.
@@ -57,4 +60,4 @@ public class GridIoPolicy {
     public static boolean isReservedGridIoPolicy(byte key) {
         return key >= 0 && key <= 31;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f3b2fcd/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java
index 59e5e7d..66efc8c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.processors.pool;
 
+import java.util.Arrays;
+import java.util.concurrent.Executor;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.GridKernalContext;
@@ -25,9 +27,6 @@ import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.plugin.IgnitePluginProcessor;
 import org.apache.ignite.plugin.extensions.communication.IoPool;
 
-import java.util.Arrays;
-import java.util.concurrent.Executor;
-
 /**
  * Processor which abstracts out thread pool management.
  */
@@ -128,6 +127,11 @@ public class PoolProcessor extends GridProcessorAdapter {
 
                 return ctx.getIgfsExecutorService();
 
+            case GridIoPolicy.SERVICE_POOL:
+                assert ctx.getServiceExecutorService() != null : "Service pool is not configured.";
+
+                return ctx.getServiceExecutorService();
+
             default: {
                 if (plc < 0)
                     throw new IgniteCheckedException("Policy cannot be negative: " + plc);

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f3b2fcd/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java
index aa60934..d2e96ba 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java
@@ -42,15 +42,19 @@ import org.apache.ignite.internal.GridClosureCallMode;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteCallable;
+import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.apache.ignite.services.Service;
 import org.jsr166.ThreadLocalRandom8;
 
+import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_IO_POLICY;
+
 /**
  * Wrapper for making {@link org.apache.ignite.services.Service} class proxies.
  */
@@ -58,6 +62,9 @@ public class GridServiceProxy<T> implements Serializable {
     /** */
     private static final long serialVersionUID = 0L;
 
+    /** */
+    private static final IgniteProductVersion SVC_POOL_SINCE_VER = IgniteProductVersion.fromString("1.8.5");
+
     /** Grid logger. */
     @GridToStringExclude
     private final IgniteLogger log;
@@ -176,6 +183,9 @@ public class GridServiceProxy<T> implements Serializable {
                         }
                     }
                     else {
+                        if (node.version().compareTo(SVC_POOL_SINCE_VER) >= 0)
+                            ctx.task().setThreadContext(TC_IO_POLICY, GridIoPolicy.SERVICE_POOL);
+
                         // Execute service remotely.
                         return ctx.closure().callAsyncNoFailover(
                             GridClosureCallMode.BROADCAST,

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f3b2fcd/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskThreadContextKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskThreadContextKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskThreadContextKey.java
index 3bb1924..2ec63df 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskThreadContextKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskThreadContextKey.java
@@ -34,5 +34,8 @@ public enum GridTaskThreadContextKey {
     TC_TIMEOUT,
 
     /** Security subject ID. */
-    TC_SUBJ_ID
-}
\ No newline at end of file
+    TC_SUBJ_ID,
+
+    /** IO manager policy. */
+    TC_IO_POLICY
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f3b2fcd/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
index d89e80b..c426008 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
@@ -101,6 +101,7 @@ import static org.apache.ignite.internal.GridTopic.TOPIC_JOB;
 import static org.apache.ignite.internal.GridTopic.TOPIC_JOB_CANCEL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.MANAGEMENT_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.PUBLIC_POOL;
+import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_IO_POLICY;
 import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_NO_FAILOVER;
 
 /**
@@ -1381,8 +1382,21 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
                     if (loc)
                         ctx.job().processJobExecuteRequest(ctx.discovery().localNode(), req);
                     else {
+                        byte plc;
+
+                        if (internal)
+                            plc = MANAGEMENT_POOL;
+                        else {
+                            Byte ctxPlc = getThreadContext(TC_IO_POLICY);
+
+                            if (ctxPlc != null)
+                                plc = ctxPlc;
+                            else
+                                plc = PUBLIC_POOL;
+                        }
+
                         // Send job execution request.
-                        ctx.io().send(node, TOPIC_JOB, req, internal ? MANAGEMENT_POOL : PUBLIC_POOL);
+                        ctx.io().send(node, TOPIC_JOB, req, plc);
 
                         if (log.isDebugEnabled())
                             log.debug("Sent job request [req=" + req + ", node=" + node + ']');

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f3b2fcd/modules/core/src/test/java/org/apache/ignite/services/ServiceThreadPoolSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/services/ServiceThreadPoolSelfTest.java b/modules/core/src/test/java/org/apache/ignite/services/ServiceThreadPoolSelfTest.java
new file mode 100644
index 0000000..40efdfa
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/services/ServiceThreadPoolSelfTest.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.services;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Test verifying that services thread pool is properly used.
+ */
+public class ServiceThreadPoolSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDefaultPoolSize() throws Exception {
+        Ignite ignite = startGrid("grid", new IgniteConfiguration());
+
+        IgniteConfiguration cfg = ignite.configuration();
+
+        assertEquals(IgniteConfiguration.DFLT_PUBLIC_THREAD_CNT, cfg.getPublicThreadPoolSize());
+        assertEquals(IgniteConfiguration.DFLT_PUBLIC_THREAD_CNT, cfg.getServiceThreadPoolSize());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInheritedPoolSize() throws Exception {
+        Ignite ignite = startGrid("grid", new IgniteConfiguration().setPublicThreadPoolSize(42));
+
+        IgniteConfiguration cfg = ignite.configuration();
+
+        assertEquals(42, cfg.getPublicThreadPoolSize());
+        assertEquals(42, cfg.getServiceThreadPoolSize());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCustomPoolSize() throws Exception {
+        Ignite ignite = startGrid("grid", new IgniteConfiguration().setServiceThreadPoolSize(42));
+
+        IgniteConfiguration cfg = ignite.configuration();
+
+        assertEquals(IgniteConfiguration.DFLT_PUBLIC_THREAD_CNT, cfg.getPublicThreadPoolSize());
+        assertEquals(42, cfg.getServiceThreadPoolSize());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testExecution() throws Exception {
+        startGrid(0); // Server.
+
+        Ignition.setClientMode(true);
+
+        Ignite ignite = startGrid(); // Client.
+
+        ignite.services().deployClusterSingleton("my-service", new MyServiceImpl());
+
+        MyService svc = ignite.services().serviceProxy("my-service", MyService.class, false);
+
+        svc.hello();
+    }
+
+    /**
+     */
+    private interface MyService extends Service{
+        /**
+         * Hello!
+         */
+        void hello();
+    }
+
+    /**
+     */
+    private static class MyServiceImpl implements MyService {
+        /** {@inheritDoc} */
+        @Override public void hello() {
+            String thread = Thread.currentThread().getName();
+
+            assertTrue("Service is executed in wrong thread: " + thread, thread.startsWith("svc-#"));
+        }
+
+        /** {@inheritDoc} */
+        @Override public void cancel(ServiceContext ctx) {
+        }
+
+        /** {@inheritDoc} */
+        @Override public void init(ServiceContext ctx) throws Exception {
+        }
+
+        /** {@inheritDoc} */
+        @Override public void execute(ServiceContext ctx) throws Exception {
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f3b2fcd/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
index 143159d..49cb206 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
@@ -64,6 +64,7 @@ public class GridTestKernalContext extends GridKernalContextImpl {
                 null,
                 null,
                 null,
+                null,
                 U.allPluginProviders()
         );
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f3b2fcd/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
index 4f771d6..9b41fbc 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
@@ -72,6 +72,7 @@ import org.apache.ignite.internal.processors.service.IgniteServiceProxyTimeoutIn
 import org.apache.ignite.internal.processors.service.IgniteServiceReassignmentTest;
 import org.apache.ignite.internal.processors.service.ServicePredicateAccessCacheTest;
 import org.apache.ignite.internal.util.GridStartupWithUndefinedIgniteHomeSelfTest;
+import org.apache.ignite.services.ServiceThreadPoolSelfTest;
 import org.apache.ignite.spi.communication.GridCacheMessageSelfTest;
 import org.apache.ignite.testframework.GridTestUtils;
 
@@ -147,6 +148,7 @@ public class IgniteKernalSelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteServiceProxyTimeoutInitializedTest.class);
         suite.addTestSuite(IgniteServiceDynamicCachesSelfTest.class);
         suite.addTestSuite(GridServiceContinuousQueryRedeploy.class);
+        suite.addTestSuite(ServiceThreadPoolSelfTest.class);
 
         suite.addTestSuite(IgniteServiceDeploymentClassLoadingDefaultMarshallerTest.class);
         suite.addTestSuite(IgniteServiceDeploymentClassLoadingOptimizedMarshallerTest.class);