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/11/17 10:51:55 UTC

[32/50] [abbrv] ignite git commit: ignite-3812 added thread pool size validation

ignite-3812 added thread pool size validation


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

Branch: refs/heads/ignite-2693
Commit: 2bc234ed8bec13998fdb2f6eeb4e8703263be21c
Parents: 51e96d6
Author: Alexandr Kuramshin <ak...@gridgain.com>
Authored: Wed Nov 9 16:15:01 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Nov 9 16:15:01 2016 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/IgnitionEx.java  |  32 ++++-
 .../testsuites/IgniteUtilSelfTestSuite.java     |   2 +
 .../ignite/thread/IgniteThreadPoolSizeTest.java | 131 +++++++++++++++++++
 3 files changed, 164 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2bc234ed/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 5b2c3fc..b3a9eec 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
@@ -1640,6 +1640,8 @@ public class IgnitionEx {
                 ensureMultiInstanceSupport(myCfg.getSwapSpaceSpi());
             }
 
+            validateThreadPoolSize(cfg.getPublicThreadPoolSize(), "public");
+
             execSvc = new IgniteThreadPoolExecutor(
                 "pub",
                 cfg.getGridName(),
@@ -1652,6 +1654,8 @@ public class IgnitionEx {
 
             // Note that since we use 'LinkedBlockingQueue', number of
             // maximum threads has no effect.
+            validateThreadPoolSize(cfg.getSystemThreadPoolSize(), "system");
+
             sysExecSvc = new IgniteThreadPoolExecutor(
                 "sys",
                 cfg.getGridName(),
@@ -1666,6 +1670,8 @@ public class IgnitionEx {
             // maximum threads has no effect.
             // Note, that we do not pre-start threads here as management pool may
             // not be needed.
+            validateThreadPoolSize(cfg.getManagementThreadPoolSize(), "management");
+
             mgmtExecSvc = new IgniteThreadPoolExecutor(
                 "mgmt",
                 cfg.getGridName(),
@@ -1680,6 +1686,7 @@ public class IgnitionEx {
             // maximum threads has no effect.
             // Note, that we do not pre-start threads here as class loading pool may
             // not be needed.
+            validateThreadPoolSize(cfg.getPeerClassLoadingThreadPoolSize(), "peer class loading");
             p2pExecSvc = new IgniteThreadPoolExecutor(
                 "p2p",
                 cfg.getGridName(),
@@ -1691,6 +1698,8 @@ public class IgnitionEx {
             p2pExecSvc.allowCoreThreadTimeOut(true);
 
             // Note that we do not pre-start threads here as igfs pool may not be needed.
+            validateThreadPoolSize(cfg.getIgfsThreadPoolSize(), "IGFS");
+
             igfsExecSvc = new IgniteThreadPoolExecutor(
                 cfg.getIgfsThreadPoolSize(),
                 cfg.getIgfsThreadPoolSize(),
@@ -1702,12 +1711,16 @@ public class IgnitionEx {
             igfsExecSvc.allowCoreThreadTimeOut(true);
 
             // Note that we do not pre-start threads here as this pool may not be needed.
+            validateThreadPoolSize(cfg.getAsyncCallbackPoolSize(), "async callback");
+
             callbackExecSvc = new IgniteStripedThreadPoolExecutor(
                 cfg.getAsyncCallbackPoolSize(),
                 cfg.getGridName(),
                 "callback");
 
             if (myCfg.getConnectorConfiguration() != null) {
+                validateThreadPoolSize(myCfg.getConnectorConfiguration().getThreadPoolSize(), "connector");
+
                 restExecSvc = new IgniteThreadPoolExecutor(
                     "rest",
                     myCfg.getGridName(),
@@ -1720,6 +1733,8 @@ public class IgnitionEx {
                 restExecSvc.allowCoreThreadTimeOut(true);
             }
 
+            validateThreadPoolSize(myCfg.getUtilityCacheThreadPoolSize(), "utility cache");
+
             utilityCacheExecSvc = new IgniteThreadPoolExecutor(
                 "utility",
                 cfg.getGridName(),
@@ -1730,6 +1745,8 @@ public class IgnitionEx {
 
             utilityCacheExecSvc.allowCoreThreadTimeOut(true);
 
+            validateThreadPoolSize(myCfg.getMarshallerCacheThreadPoolSize(), "marshaller cache");
+
             marshCacheExecSvc = new IgniteThreadPoolExecutor(
                 "marshaller-cache",
                 cfg.getGridName(),
@@ -1838,6 +1855,19 @@ public class IgnitionEx {
         }
 
         /**
+         * @param poolSize an actual value in the configuration.
+         * @param poolName a name of the pool like 'management'.
+         * @throws IgniteCheckedException If the poolSize is wrong.
+         */
+        private static void validateThreadPoolSize(int poolSize, String poolName)
+            throws IgniteCheckedException {
+            if (poolSize <= 0) {
+                throw new IgniteCheckedException("Invalid " + poolName + " thread pool size" +
+                    " (must be greater than 0), actual value: " + poolSize);
+            }
+        }
+
+        /**
          * @param cfg Ignite configuration copy to.
          * @return New ignite configuration.
          * @throws IgniteCheckedException If failed.
@@ -2019,7 +2049,7 @@ public class IgnitionEx {
             if (userCaches != null && userCaches.length > 0) {
                 if (!U.discoOrdered(cfg.getDiscoverySpi()) && !U.relaxDiscoveryOrdered())
                     throw new IgniteCheckedException("Discovery SPI implementation does not support node ordering and " +
-                        "cannot be used with cache (use SPI with @GridDiscoverySpiOrderSupport annotation, " +
+                        "cannot be used with cache (use SPI with @DiscoverySpiOrderSupport annotation, " +
                         "like TcpDiscoverySpi)");
 
                 for (CacheConfiguration ccfg : userCaches) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/2bc234ed/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
index bd7bb96..078b865 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
@@ -36,6 +36,7 @@ import org.apache.ignite.lang.GridByteArrayListSelfTest;
 import org.apache.ignite.spi.discovery.ClusterMetricsSnapshotSerializeSelfTest;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.thread.GridThreadPoolExecutorServiceSelfTest;
+import org.apache.ignite.thread.IgniteThreadPoolSizeTest;
 import org.apache.ignite.util.GridLongListSelfTest;
 import org.apache.ignite.util.GridMessageCollectionTest;
 import org.apache.ignite.util.GridQueueSelfTest;
@@ -64,6 +65,7 @@ public class IgniteUtilSelfTestSuite extends TestSuite {
         TestSuite suite = new TestSuite("Ignite Util Test Suite");
 
         suite.addTestSuite(GridThreadPoolExecutorServiceSelfTest.class);
+        suite.addTestSuite(IgniteThreadPoolSizeTest.class);
         GridTestUtils.addTestIfNeeded(suite, IgniteUtilsSelfTest.class, ignoredTests);
         suite.addTestSuite(GridSpinReadWriteLockSelfTest.class);
         suite.addTestSuite(GridQueueSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/2bc234ed/modules/core/src/test/java/org/apache/ignite/thread/IgniteThreadPoolSizeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/thread/IgniteThreadPoolSizeTest.java b/modules/core/src/test/java/org/apache/ignite/thread/IgniteThreadPoolSizeTest.java
new file mode 100644
index 0000000..d1fd4e7
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/thread/IgniteThreadPoolSizeTest.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.thread;
+
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class IgniteThreadPoolSizeTest extends GridCommonAbstractTest {
+    /** Wrong thread pool size value for testing */
+    private static final int WRONG_VALUE = 0;
+
+    /**
+     * @return Ignite configuration.
+     */
+    private IgniteConfiguration configuration() {
+        return new IgniteConfiguration();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAsyncCallbackPoolSize() throws Exception {
+        testWrongPoolSize(configuration().setAsyncCallbackPoolSize(WRONG_VALUE));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIgfsThreadPoolSize() throws Exception {
+        testWrongPoolSize(configuration().setIgfsThreadPoolSize(WRONG_VALUE));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testManagementThreadPoolSize() throws Exception {
+        testWrongPoolSize(configuration().setManagementThreadPoolSize(WRONG_VALUE));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPeerClassLoadingThreadPoolSize() throws Exception {
+        testWrongPoolSize(configuration().setPeerClassLoadingThreadPoolSize(WRONG_VALUE));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPublicThreadPoolSize() throws Exception {
+        testWrongPoolSize(configuration().setPublicThreadPoolSize(WRONG_VALUE));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRebalanceThreadPoolSize() throws Exception {
+        testWrongPoolSize(configuration().setRebalanceThreadPoolSize(WRONG_VALUE));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSystemThreadPoolSize() throws Exception {
+        testWrongPoolSize(configuration().setSystemThreadPoolSize(WRONG_VALUE));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testUtilityCachePoolSize() throws Exception {
+        testWrongPoolSize(configuration().setUtilityCachePoolSize(WRONG_VALUE));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("deprecated")
+    public void testMarshallerCachePoolSize() throws Exception {
+        testWrongPoolSize(configuration().setMarshallerCachePoolSize(WRONG_VALUE));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConnectorThreadPoolSize() throws Exception {
+        final IgniteConfiguration cfg = configuration();
+
+        cfg.getConnectorConfiguration().setThreadPoolSize(WRONG_VALUE);
+
+        testWrongPoolSize(cfg);
+    }
+
+    /**
+     * Performs testing for wrong tread pool size.
+     *
+     * @param cfg an IgniteConfiguration with the only one thread pool size assigned with the WRONG_VALUE.
+     * @throws Exception If failed.
+     */
+    private void testWrongPoolSize(IgniteConfiguration cfg) throws Exception {
+        try {
+            Ignition.start(cfg);
+
+            fail();
+        }
+        catch (IgniteException ex) {
+            assertNotNull(ex.getMessage());
+            assertTrue(ex.getMessage().contains("thread pool size"));
+        }
+    }
+}