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/12/28 14:27:56 UTC

ignite git commit: ignite-7217 Custom thread pools monitoring added

Repository: ignite
Updated Branches:
  refs/heads/master c7b4201e8 -> df3c4df3d


ignite-7217 Custom thread pools monitoring added

Signed-off-by: Andrey Gura <ag...@apache.org>


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

Branch: refs/heads/master
Commit: df3c4df3d895c138434a97dbc5a1a580ead5799d
Parents: c7b4201
Author: Stanislav Lukyanov <st...@gmail.com>
Authored: Thu Dec 28 17:21:47 2017 +0300
Committer: Andrey Gura <ag...@apache.org>
Committed: Thu Dec 28 17:27:34 2017 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    | 499 +++++++++----------
 .../apache/ignite/internal/GridMBeansTest.java  |  86 ++++
 .../internal/GridNodeMetricsLogSelfTest.java    |  29 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +
 4 files changed, 336 insertions(+), 280 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/df3c4df3/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 e0c1730..c62d59c 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
@@ -37,10 +37,12 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Date;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.ListIterator;
 import java.util.Map;
 import java.util.Properties;
+import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ThreadPoolExecutor;
@@ -88,7 +90,6 @@ import org.apache.ignite.configuration.AtomicConfiguration;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.CollectionConfiguration;
-import org.apache.ignite.configuration.ConnectorConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.MemoryConfiguration;
@@ -282,6 +283,10 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     @GridToStringExclude
     private GridKernalContextImpl ctx;
 
+    /** Helper that registers MBeans */
+    @GridToStringExclude
+    private final MBeansManager mBeansMgr = new MBeansManager();
+
     /** Configuration. */
     private IgniteConfiguration cfg;
 
@@ -293,50 +298,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     /** */
     private String igniteInstanceName;
 
-    /** */
-    @GridToStringExclude
-    private ObjectName kernalMBean;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName locNodeMBean;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName allNodesMBean;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName pubExecSvcMBean;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName sysExecSvcMBean;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName mgmtExecSvcMBean;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName p2PExecSvcMBean;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName restExecSvcMBean;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName qryExecSvcMBean;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName schemaExecSvcMBean;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName stripedExecSvcMBean;
-
     /** Kernal start timestamp. */
     private long startTime = U.currentTimeMillis();
 
@@ -753,7 +714,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         IgniteStripedThreadPoolExecutor callbackExecSvc,
         ExecutorService qryExecSvc,
         ExecutorService schemaExecSvc,
-        Map<String, ? extends ExecutorService> customExecSvcs,
+        @Nullable final Map<String, ? extends ExecutorService> customExecSvcs,
         GridAbsClosure errHnd
     )
         throws IgniteCheckedException
@@ -1083,12 +1044,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                 reconnectState.waitFirstReconnect();
 
             // Register MBeans.
-            registerKernalMBean();
-            registerClusterMetricsMBeans();
-            registerExecutorMBeans(execSvc, sysExecSvc, p2pExecSvc, mgmtExecSvc, restExecSvc, qryExecSvc,
-                schemaExecSvc);
-
-            registerStripedExecutorMBean(stripedExecSvc);
+            mBeansMgr.registerAllMBeans(utilityCachePool, execSvc, svcExecSvc, sysExecSvc, stripedExecSvc, p2pExecSvc,
+                mgmtExecSvc, igfsExecSvc, dataStreamExecSvc, restExecSvc, affExecSvc, idxExecSvc, callbackExecSvc,
+                qryExecSvc, schemaExecSvc, customExecSvcs);
 
             // Lifecycle bean notifications.
             notifyLifecycleBeans(AFTER_NODE_START);
@@ -1227,34 +1185,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                                 // No-op.
                             }
 
-                            int pubPoolActiveThreads = 0;
-                            int pubPoolIdleThreads = 0;
-                            int pubPoolQSize = 0;
-
-                            if (execSvc instanceof ThreadPoolExecutor) {
-                                ThreadPoolExecutor exec = (ThreadPoolExecutor)execSvc;
-
-                                int poolSize = exec.getPoolSize();
-
-                                pubPoolActiveThreads = Math.min(poolSize, exec.getActiveCount());
-                                pubPoolIdleThreads = poolSize - pubPoolActiveThreads;
-                                pubPoolQSize = exec.getQueue().size();
-                            }
-
-                            int sysPoolActiveThreads = 0;
-                            int sysPoolIdleThreads = 0;
-                            int sysPoolQSize = 0;
-
-                            if (sysExecSvc instanceof ThreadPoolExecutor) {
-                                ThreadPoolExecutor exec = (ThreadPoolExecutor)sysExecSvc;
-
-                                int poolSize = exec.getPoolSize();
-
-                                sysPoolActiveThreads = Math.min(poolSize, exec.getActiveCount());
-                                sysPoolIdleThreads = poolSize - sysPoolActiveThreads;
-                                sysPoolQSize = exec.getQueue().size();
-                            }
-
                             int loadedPages = 0;
 
                             Collection<DataRegion> policies = ctx.cache().context().database().dataRegions();
@@ -1278,11 +1208,20 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                                 dblFmt.format(freeHeapPct) + "%, comm=" + dblFmt.format(heapCommInMBytes) + "MB]" + NL +
                                 "    ^-- Non heap [used=" + dblFmt.format(nonHeapUsedInMBytes) + "MB, free=" +
                                 dblFmt.format(freeNonHeapPct) + "%, comm=" + dblFmt.format(nonHeapCommInMBytes) + "MB]" + NL +
-                                "    ^-- Public thread pool [active=" + pubPoolActiveThreads + ", idle=" +
-                                pubPoolIdleThreads + ", qSize=" + pubPoolQSize + "]" + NL +
-                                "    ^-- System thread pool [active=" + sysPoolActiveThreads + ", idle=" +
-                                sysPoolIdleThreads + ", qSize=" + sysPoolQSize + "]" + NL +
-                                "    ^-- Outbound messages queue [size=" + m.getOutboundMessagesQueueSize() + "]";
+                                "    ^-- Outbound messages queue [size=" + m.getOutboundMessagesQueueSize() + "]" + NL +
+                                "    ^-- " + createExecutorDescription("Public thread pool", execSvc) + NL +
+                                "    ^-- " + createExecutorDescription("System thread pool", sysExecSvc);
+
+                            if (customExecSvcs != null) {
+                                StringBuilder customSvcsMsg = new StringBuilder();
+
+                                for (Map.Entry<String, ? extends ExecutorService> entry : customExecSvcs.entrySet()) {
+                                    customSvcsMsg.append(NL).append("    ^-- ")
+                                        .append(createExecutorDescription(entry.getKey(), entry.getValue()));
+                                }
+
+                                msg = msg + customSvcsMsg;
+                            }
 
                             if (log.isInfoEnabled())
                                 log.info(msg);
@@ -1324,6 +1263,30 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     }
 
     /**
+     * Create description of an executor service for logging.
+     *
+     * @param execSvcName name of the service
+     * @param execSvc service to create a description for
+     */
+    private String createExecutorDescription(String execSvcName, ExecutorService execSvc) {
+        int poolActiveThreads = 0;
+        int poolIdleThreads = 0;
+        int poolQSize = 0;
+
+        if (execSvc instanceof ThreadPoolExecutor) {
+            ThreadPoolExecutor exec = (ThreadPoolExecutor)execSvc;
+
+            int poolSize = exec.getPoolSize();
+
+            poolActiveThreads = Math.min(poolSize, exec.getActiveCount());
+            poolIdleThreads = poolSize - poolActiveThreads;
+            poolQSize = exec.getQueue().size();
+        }
+
+        return execSvcName + " [active=" + poolActiveThreads + ", idle=" + poolIdleThreads + ", qSize=" + poolQSize + "]";
+    }
+
+    /**
      * Create Hadoop component.
      *
      * @return Non-null Hadoop component: workable or no-op.
@@ -1677,183 +1640,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         }
     }
 
-    /** @throws IgniteCheckedException If registration failed. */
-    private void registerKernalMBean() throws IgniteCheckedException {
-        if(U.IGNITE_MBEANS_DISABLED)
-            return;
-
-        try {
-            kernalMBean = U.registerMBean(
-                cfg.getMBeanServer(),
-                cfg.getIgniteInstanceName(),
-                "Kernal",
-                getClass().getSimpleName(),
-                this,
-                IgniteMXBean.class);
-
-            if (log.isDebugEnabled())
-                log.debug("Registered kernal MBean: " + kernalMBean);
-        }
-        catch (JMException e) {
-            kernalMBean = null;
-
-            throw new IgniteCheckedException("Failed to register kernal MBean.", e);
-        }
-    }
-
-    /**
-     * Register instance of ClusterMetricsMBean.
-     *
-     * @param mbean MBean instance to register.
-     * @throws IgniteCheckedException If registration failed.
-     */
-    private ObjectName registerClusterMetricsMBean(ClusterMetricsMXBean mbean) throws IgniteCheckedException {
-        if(U.IGNITE_MBEANS_DISABLED)
-            return null;
-
-        ObjectName objName;
-
-        try {
-            objName = U.registerMBean(
-                cfg.getMBeanServer(),
-                cfg.getIgniteInstanceName(),
-                "Kernal",
-                mbean.getClass().getSimpleName(),
-                mbean,
-                ClusterMetricsMXBean.class);
-
-            if (log.isDebugEnabled())
-                log.debug("Registered MBean: " + objName);
-
-            return objName;
-        }
-        catch (JMException e) {
-            throw new IgniteCheckedException("Failed to register MBean: " + mbean.getClass().getSimpleName(), e);
-        }
-    }
-
-    /** @throws IgniteCheckedException If registration failed. */
-    private void registerClusterMetricsMBeans() throws IgniteCheckedException {
-        locNodeMBean = registerClusterMetricsMBean(new ClusterLocalNodeMetricsMXBeanImpl(ctx.discovery()));
-        allNodesMBean = registerClusterMetricsMBean(new ClusterMetricsMXBeanImpl(cluster()));
-    }
-
-    /**
-     * @param execSvc Public executor service.
-     * @param sysExecSvc System executor service.
-     * @param p2pExecSvc P2P executor service.
-     * @param mgmtExecSvc Management executor service.
-     * @param restExecSvc Query executor service.
-     * @param schemaExecSvc Schema executor service.
-     * @throws IgniteCheckedException If failed.
-     */
-    private void registerExecutorMBeans(ExecutorService execSvc,
-        ExecutorService sysExecSvc,
-        ExecutorService p2pExecSvc,
-        ExecutorService mgmtExecSvc,
-        ExecutorService restExecSvc,
-        ExecutorService qryExecSvc,
-        ExecutorService schemaExecSvc
-    ) throws IgniteCheckedException {if(U.IGNITE_MBEANS_DISABLED)
-            return;
-        pubExecSvcMBean = registerExecutorMBean(execSvc, "GridExecutionExecutor");
-        sysExecSvcMBean = registerExecutorMBean(sysExecSvc, "GridSystemExecutor");
-        mgmtExecSvcMBean = registerExecutorMBean(mgmtExecSvc, "GridManagementExecutor");
-        p2PExecSvcMBean = registerExecutorMBean(p2pExecSvc, "GridClassLoadingExecutor");
-        qryExecSvcMBean = registerExecutorMBean(qryExecSvc, "GridQueryExecutor");
-        schemaExecSvcMBean = registerExecutorMBean(schemaExecSvc, "GridSchemaExecutor");
-
-        ConnectorConfiguration clientCfg = cfg.getConnectorConfiguration();
-
-        if (clientCfg != null)
-            restExecSvcMBean = registerExecutorMBean(restExecSvc, "GridRestExecutor");
-    }
-
-    /**
-     * @param exec Executor service to register.
-     * @param name Property name for executor.
-     * @return Name for created MBean.
-     * @throws IgniteCheckedException If registration failed.
-     */
-    private ObjectName registerExecutorMBean(ExecutorService exec, String name) throws IgniteCheckedException {
-        assert exec != null;
-        assert !U.IGNITE_MBEANS_DISABLED;
-
-        try {
-            ObjectName res = U.registerMBean(
-                cfg.getMBeanServer(),
-                cfg.getIgniteInstanceName(),
-                "Thread Pools",
-                name,
-                new ThreadPoolMXBeanAdapter(exec),
-                ThreadPoolMXBean.class);
-
-            if (log.isDebugEnabled())
-                log.debug("Registered executor service MBean: " + res);
-
-            return res;
-        }
-        catch (JMException e) {
-            throw new IgniteCheckedException("Failed to register executor service MBean [name=" + name +
-                ", exec=" + exec + ']', e);
-        }
-    }
-
-    /**
-     * @param stripedExecSvc Executor service.
-     * @throws IgniteCheckedException If registration failed.
-     */
-    private void registerStripedExecutorMBean(StripedExecutor stripedExecSvc) throws IgniteCheckedException {
-        if (stripedExecSvc == null || U.IGNITE_MBEANS_DISABLED)
-            return;
-
-        String name = "StripedExecutor";
-
-        try {
-            stripedExecSvcMBean = U.registerMBean(
-                cfg.getMBeanServer(),
-                cfg.getIgniteInstanceName(),
-                "Thread Pools",
-                name,
-                new StripedExecutorMXBeanAdapter(stripedExecSvc),
-                StripedExecutorMXBean.class);
-
-            if (log.isDebugEnabled())
-                log.debug("Registered executor service MBean: " + stripedExecSvcMBean);
-        }
-        catch (JMException e) {
-            throw new IgniteCheckedException("Failed to register executor service MBean [name="
-                + name + ", exec=" + stripedExecSvc + ']', e);
-        }
-    }
-
-    /**
-     * Unregisters given mbean.
-     *
-     * @param mbean MBean to unregister.
-     * @return {@code True} if successfully unregistered, {@code false} otherwise.
-     */
-    private boolean unregisterMBean(@Nullable ObjectName mbean) {
-        if (mbean == null)
-            return true;
-
-        assert !U.IGNITE_MBEANS_DISABLED;
-
-        try {
-            cfg.getMBeanServer().unregisterMBean(mbean);
-
-            if (log.isDebugEnabled())
-                log.debug("Unregistered MBean: " + mbean);
-
-            return true;
-        }
-        catch (JMException e) {
-            U.error(log, "Failed to unregister MBean.", e);
-
-            return false;
-        }
-    }
-
     /**
      * @param mgr Manager to start.
      * @throws IgniteCheckedException Throw in case of any errors.
@@ -2292,20 +2078,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                 cache.blockGateways();
 
             // Unregister MBeans.
-            if (!(
-                unregisterMBean(pubExecSvcMBean) &
-                    unregisterMBean(sysExecSvcMBean) &
-                    unregisterMBean(mgmtExecSvcMBean) &
-                    unregisterMBean(p2PExecSvcMBean) &
-                    unregisterMBean(kernalMBean) &
-                    unregisterMBean(locNodeMBean) &
-                    unregisterMBean(allNodesMBean) &
-                    unregisterMBean(restExecSvcMBean) &
-                    unregisterMBean(qryExecSvcMBean) &
-                    unregisterMBean(schemaExecSvcMBean) &
-                    unregisterMBean(stripedExecSvcMBean)
-            ))
-                errOnStop = false;
+            if (!mBeansMgr.unregisterAllMBeans())
+                errOnStop = true;
 
             // Stop components in reverse order.
             for (ListIterator<GridComponent> it = comps.listIterator(comps.size()); it.hasPrevious(); ) {
@@ -4193,6 +3967,179 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         }
     }
 
+    /**
+     * Class that registers and unregisters MBeans for kernal.
+     */
+    private class MBeansManager {
+        /** MBean names stored to be unregistered later. */
+        private final Set<ObjectName> mBeanNames = new HashSet<>();
+
+        /**
+         * Registers all kernal MBeans (for kernal, metrics, thread pools).
+         *
+         * @param utilityCachePool Utility cache pool
+         * @param execSvc Executor service
+         * @param sysExecSvc System executor service
+         * @param stripedExecSvc Striped executor
+         * @param p2pExecSvc P2P executor service
+         * @param mgmtExecSvc Management executor service
+         * @param igfsExecSvc IGFS executor service
+         * @param dataStreamExecSvc data stream executor service
+         * @param restExecSvc Reset executor service
+         * @param affExecSvc Affinity executor service
+         * @param idxExecSvc Indexing executor service
+         * @param callbackExecSvc Callback executor service
+         * @param qryExecSvc Query executor service
+         * @param schemaExecSvc Schema executor service
+         * @param customExecSvcs Custom named executors
+         *
+         * @throws IgniteCheckedException if fails to register any of the MBeans
+         */
+        private void registerAllMBeans(
+            ExecutorService utilityCachePool,
+            final ExecutorService execSvc,
+            final ExecutorService svcExecSvc,
+            final ExecutorService sysExecSvc,
+            final StripedExecutor stripedExecSvc,
+            ExecutorService p2pExecSvc,
+            ExecutorService mgmtExecSvc,
+            ExecutorService igfsExecSvc,
+            StripedExecutor dataStreamExecSvc,
+            ExecutorService restExecSvc,
+            ExecutorService affExecSvc,
+            @Nullable ExecutorService idxExecSvc,
+            IgniteStripedThreadPoolExecutor callbackExecSvc,
+            ExecutorService qryExecSvc,
+            ExecutorService schemaExecSvc,
+            @Nullable final Map<String, ? extends ExecutorService> customExecSvcs
+        ) throws IgniteCheckedException {
+            if (U.IGNITE_MBEANS_DISABLED)
+                return;
+
+            // Kernal
+            registerMBean("Kernal", IgniteKernal.class.getSimpleName(), IgniteKernal.this, IgniteMXBean.class);
+
+            // Metrics
+            ClusterMetricsMXBean locMetricsBean = new ClusterLocalNodeMetricsMXBeanImpl(ctx.discovery());
+            registerMBean("Kernal", locMetricsBean.getClass().getSimpleName(), locMetricsBean, ClusterMetricsMXBean.class);
+            ClusterMetricsMXBean metricsBean = new ClusterMetricsMXBeanImpl(cluster());
+            registerMBean("Kernal", metricsBean.getClass().getSimpleName(), metricsBean, ClusterMetricsMXBean.class);
+
+            // Executors
+            registerExecutorMBean("GridUtilityCacheExecutor", utilityCachePool);
+            registerExecutorMBean("GridExecutionExecutor", execSvc);
+            registerExecutorMBean("GridServicesExecutor", svcExecSvc);
+            registerExecutorMBean("GridSystemExecutor", sysExecSvc);
+            registerExecutorMBean("GridClassLoadingExecutor", p2pExecSvc);
+            registerExecutorMBean("GridManagementExecutor", mgmtExecSvc);
+            registerExecutorMBean("GridIgfsExecutor", igfsExecSvc);
+            registerExecutorMBean("GridDataStreamExecutor", dataStreamExecSvc);
+            registerExecutorMBean("GridAffinityExecutor", affExecSvc);
+            registerExecutorMBean("GridCallbackExecutor", callbackExecSvc);
+            registerExecutorMBean("GridQueryExecutor", qryExecSvc);
+            registerExecutorMBean("GridSchemaExecutor", schemaExecSvc);
+
+            if (idxExecSvc != null)
+                registerExecutorMBean("GridIndexingExecutor", idxExecSvc);
+
+            if (cfg.getConnectorConfiguration() != null)
+                registerExecutorMBean("GridRestExecutor", restExecSvc);
+
+            if (stripedExecSvc != null) {
+                // striped executor uses a custom adapter
+                registerMBean("Thread Pools",
+                    "StripedExecutor",
+                    new StripedExecutorMXBeanAdapter(stripedExecSvc),
+                    StripedExecutorMXBean.class);
+            }
+
+            if (customExecSvcs != null) {
+                for (Map.Entry<String, ? extends ExecutorService> entry : customExecSvcs.entrySet())
+                    registerExecutorMBean(entry.getKey(), entry.getValue());
+            }
+        }
+
+        /**
+         * Registers a {@link ThreadPoolMXBean} for an executor.
+         *
+         * @param name name of the bean to register
+         * @param exec executor to register a bean for
+         *
+         * @throws IgniteCheckedException if registration fails.
+         */
+        private void registerExecutorMBean(String name, ExecutorService exec) throws IgniteCheckedException {
+            registerMBean("Thread Pools", name, new ThreadPoolMXBeanAdapter(exec), ThreadPoolMXBean.class);
+        }
+
+        /**
+         * Register an Ignite MBean.
+         *
+         * @param grp bean group name
+         * @param name bean name
+         * @param impl bean implementation
+         * @param itf bean interface
+         * @param <T> bean type
+         *
+         * @throws IgniteCheckedException if registration fails
+         */
+        private <T> void registerMBean(String grp, String name, T impl, Class<T> itf) throws IgniteCheckedException {
+            assert !U.IGNITE_MBEANS_DISABLED;
+
+            try {
+                ObjectName objName = U.registerMBean(
+                    cfg.getMBeanServer(),
+                    cfg.getIgniteInstanceName(),
+                    grp, name, impl, itf);
+
+                if (log.isDebugEnabled())
+                    log.debug("Registered MBean: " + objName);
+
+                mBeanNames.add(objName);
+            }
+            catch (JMException e) {
+                throw new IgniteCheckedException("Failed to register MBean " + name, e);
+            }
+        }
+
+        /**
+         * Unregisters all previously registered MBeans.
+         *
+         * @return {@code true} if all mbeans were unregistered successfully; {@code false} otherwise.
+         */
+        private boolean unregisterAllMBeans() {
+            boolean success = true;
+
+            for (ObjectName name : mBeanNames)
+                success = success && unregisterMBean(name);
+
+            return success;
+        }
+
+        /**
+         * Unregisters given MBean.
+         *
+         * @param mbean MBean to unregister.
+         * @return {@code true} if successfully unregistered, {@code false} otherwise.
+         */
+        private boolean unregisterMBean(ObjectName mbean) {
+            assert !U.IGNITE_MBEANS_DISABLED;
+
+            try {
+                cfg.getMBeanServer().unregisterMBean(mbean);
+
+                if (log.isDebugEnabled())
+                    log.debug("Unregistered MBean: " + mbean);
+
+                return true;
+            }
+            catch (JMException e) {
+                U.error(log, "Failed to unregister MBean.", e);
+
+                return false;
+            }
+        }
+    }
+
     /** {@inheritDoc} */
     @Override public void runIoTest(
         long warmup,

http://git-wip-us.apache.org/repos/asf/ignite/blob/df3c4df3/modules/core/src/test/java/org/apache/ignite/internal/GridMBeansTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridMBeansTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridMBeansTest.java
new file mode 100644
index 0000000..1c39982
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridMBeansTest.java
@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+import org.apache.ignite.configuration.ExecutorConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import javax.management.ObjectName;
+
+/**
+ * Tests for the standard JMX beans registered by the kernal.
+ */
+public class GridMBeansTest extends GridCommonAbstractTest {
+    /** Executor name for setExecutorConfiguration */
+    private static final String CUSTOM_EXECUTOR_0 = "Custom executor 0";
+
+    /** Executor name for setExecutorConfiguration */
+    private static final String CUSTOM_EXECUTOR_1 = "Custom executor 1";
+
+    /** Create test and auto-start the grid */
+    public GridMBeansTest() {
+        super(true);
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * This implementation registers adds custom executors to the configuration.
+     */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setExecutorConfiguration(new ExecutorConfiguration(CUSTOM_EXECUTOR_0),
+            new ExecutorConfiguration(CUSTOM_EXECUTOR_1));
+
+        return cfg;
+    }
+
+    /** Check that kernal bean is available */
+    public void testKernalBeans() throws Exception {
+        checkBean("Kernal", "IgniteKernal", "InstanceName", grid().name());
+        checkBean("Kernal", "ClusterMetricsMXBeanImpl", "TotalServerNodes", 1);
+        checkBean("Kernal", "ClusterMetricsMXBeanImpl", "TotalServerNodes", 1);
+    }
+
+    /** Check that kernal bean is available */
+    public void testExecutorBeans() throws Exception {
+        // standard executors
+        checkBean("Thread Pools", "GridExecutionExecutor", "Terminated", false);
+        checkBean("Thread Pools", "GridSystemExecutor", "Terminated", false);
+        checkBean("Thread Pools", "GridManagementExecutor", "Terminated", false);
+        checkBean("Thread Pools", "GridClassLoadingExecutor", "Terminated", false);
+        checkBean("Thread Pools", "GridQueryExecutor", "Terminated", false);
+        checkBean("Thread Pools", "GridSchemaExecutor", "Terminated", false);
+        checkBean("Thread Pools", "StripedExecutor", "Terminated", false);
+
+        // custom executors
+        checkBean("Thread Pools", CUSTOM_EXECUTOR_0, "Terminated", false);
+        checkBean("Thread Pools", CUSTOM_EXECUTOR_1, "Terminated", false);
+    }
+
+    /** Checks that a bean with the specified group and name is available and has the expected attribute */
+    private void checkBean(String grp, String name, String attributeName, Object expAttributeVal) throws Exception {
+        ObjectName mBeanName = IgniteUtils.makeMBeanName(grid().name(), grp, name);
+        Object attributeVal = grid().configuration().getMBeanServer().getAttribute(mBeanName, attributeName);
+
+        assertEquals(expAttributeVal, attributeVal);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/df3c4df3/modules/core/src/test/java/org/apache/ignite/internal/GridNodeMetricsLogSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridNodeMetricsLogSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridNodeMetricsLogSelfTest.java
index b6114d1..9edebd5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridNodeMetricsLogSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridNodeMetricsLogSelfTest.java
@@ -21,6 +21,7 @@ package org.apache.ignite.internal;
 import java.io.StringWriter;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.configuration.ExecutorConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.testframework.junits.common.GridCommonTest;
@@ -35,6 +36,12 @@ import org.apache.log4j.WriterAppender;
 @SuppressWarnings({"ProhibitedExceptionDeclared"})
 @GridCommonTest(group = "Kernal")
 public class GridNodeMetricsLogSelfTest extends GridCommonAbstractTest {
+    /** Executor name for setExecutorConfiguration */
+    private static final String CUSTOM_EXECUTOR_0 = "Custom executor 0";
+
+    /** Executor name for setExecutorConfiguration */
+    private static final String CUSTOM_EXECUTOR_1 = "Custom executor 1";
+
     /** {@inheritDoc} */
     @SuppressWarnings({"unchecked"})
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
@@ -42,6 +49,9 @@ public class GridNodeMetricsLogSelfTest extends GridCommonAbstractTest {
 
         cfg.setMetricsLogFrequency(1000);
 
+        cfg.setExecutorConfiguration(new ExecutorConfiguration(CUSTOM_EXECUTOR_0),
+            new ExecutorConfiguration(CUSTOM_EXECUTOR_1));
+
         return cfg;
     }
 
@@ -85,9 +95,20 @@ public class GridNodeMetricsLogSelfTest extends GridCommonAbstractTest {
 
         Logger.getRootLogger().removeAppender(app);
 
-        assert fullLog.contains("Metrics for local node");
-        assert fullLog.contains("uptime=");
-        assert fullLog.contains("Non heap");
-        assert fullLog.contains("Outbound messages queue");
+        String msg = "Metrics are missing in the log or have an unexpected format";
+
+        // don't check the format strictly, but check that all expected metrics are present
+        assertTrue(msg, fullLog.contains("Metrics for local node (to disable set 'metricsLogFrequency' to 0)"));
+        assertTrue(msg, fullLog.matches("(?s).*Node \\[id=.*, name=.*, uptime=.*].*"));
+        assertTrue(msg, fullLog.matches("(?s).*H/N/C \\[hosts=.*, nodes=.*, CPUs=.*].*"));
+        assertTrue(msg, fullLog.matches("(?s).*CPU \\[cur=.*, avg=.*, GC=.*].*"));
+        assertTrue(msg, fullLog.matches("(?s).*PageMemory \\[pages=.*].*"));
+        assertTrue(msg, fullLog.matches("(?s).*Heap \\[used=.*, free=.*, comm=.*].*"));
+        assertTrue(msg, fullLog.matches("(?s).*Non heap \\[used=.*, free=.*, comm=.*].*"));
+        assertTrue(msg, fullLog.matches("(?s).*Outbound messages queue \\[size=.*].*"));
+        assertTrue(msg, fullLog.matches("(?s).*Public thread pool \\[active=.*, idle=.*, qSize=.*].*"));
+        assertTrue(msg, fullLog.matches("(?s).*System thread pool \\[active=.*, idle=.*, qSize=.*].*"));
+        assertTrue(msg, fullLog.matches("(?s).*" + CUSTOM_EXECUTOR_0 + " \\[active=.*, idle=.*, qSize=.*].*"));
+        assertTrue(msg, fullLog.matches("(?s).*" + CUSTOM_EXECUTOR_1 + " \\[active=.*, idle=.*, qSize=.*].*"));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/df3c4df3/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index 7cdc670..2cc2047 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -25,6 +25,7 @@ import org.apache.ignite.internal.ClusterGroupSelfTest;
 import org.apache.ignite.internal.GridFailFastNodeFailureDetectionSelfTest;
 import org.apache.ignite.internal.GridLifecycleAwareSelfTest;
 import org.apache.ignite.internal.GridLifecycleBeanSelfTest;
+import org.apache.ignite.internal.GridMBeansTest;
 import org.apache.ignite.internal.GridNodeMetricsLogSelfTest;
 import org.apache.ignite.internal.GridProjectionForCachesSelfTest;
 import org.apache.ignite.internal.GridReduceSelfTest;
@@ -149,6 +150,7 @@ public class IgniteBasicTestSuite extends TestSuite {
         suite.addTestSuite(IgniteMarshallerCacheConcurrentReadWriteTest.class);
         suite.addTestSuite(GridNodeMetricsLogSelfTest.class);
         suite.addTestSuite(GridLocalIgniteSerializationTest.class);
+        suite.addTestSuite(GridMBeansTest.class);
 
         suite.addTestSuite(IgniteExceptionInNioWorkerSelfTest.class);
         suite.addTestSuite(IgniteLocalNodeMapBeforeStartTest.class);