Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 39075200CC9 for ; Mon, 17 Jul 2017 14:27:15 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 37AAE164D8A; Mon, 17 Jul 2017 12:27:15 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id B8390164D8B for ; Mon, 17 Jul 2017 14:27:13 +0200 (CEST) Received: (qmail 2388 invoked by uid 500); 17 Jul 2017 12:27:12 -0000 Mailing-List: contact commits-help@ignite.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.apache.org Delivered-To: mailing list commits@ignite.apache.org Received: (qmail 2261 invoked by uid 99); 17 Jul 2017 12:27:12 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 17 Jul 2017 12:27:12 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 78E2CF3220; Mon, 17 Jul 2017 12:27:10 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: agoncharuk@apache.org To: commits@ignite.apache.org Date: Mon, 17 Jul 2017 12:27:11 -0000 Message-Id: In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [02/12] ignite git commit: IGNITE-4831: Add an option to disable MBeans. This closes #2265. archived-at: Mon, 17 Jul 2017 12:27:15 -0000 IGNITE-4831: Add an option to disable MBeans. This closes #2265. (cherry picked from commit 7283edb) Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/9fa78849 Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/9fa78849 Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/9fa78849 Branch: refs/heads/ignite-5757 Commit: 9fa78849c1ecc45fa56f518ad229e76725c27790 Parents: 1e7e692 Author: Andrey V. Mashenkov Authored: Fri Jul 14 14:12:29 2017 +0300 Committer: Andrey V. Mashenkov Committed: Fri Jul 14 14:29:25 2017 +0300 ---------------------------------------------------------------------- .../apache/ignite/IgniteSystemProperties.java | 8 ++ .../org/apache/ignite/cache/CacheManager.java | 10 ++ .../apache/ignite/internal/IgniteKernal.java | 71 ++++++----- .../org/apache/ignite/internal/IgnitionEx.java | 10 +- .../client/router/impl/GridTcpRouterImpl.java | 57 ++++++--- .../processors/cache/GridCacheProcessor.java | 11 +- .../GridCacheDatabaseSharedManager.java | 28 +++-- .../IgniteCacheDatabaseSharedManager.java | 44 ++++--- .../ignite/internal/util/IgniteUtils.java | 23 +++- .../org/apache/ignite/spi/IgniteSpiAdapter.java | 10 +- .../testsuites/IgniteUtilSelfTestSuite.java | 2 + .../util/mbeans/GridMBeanDisableSelfTest.java | 121 +++++++++++++++++++ 12 files changed, 312 insertions(+), 83 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/9fa78849/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index 35b0577..1a2887a 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -404,6 +404,14 @@ public final class IgniteSystemProperties { public static final String IGNITE_MBEAN_APPEND_CLASS_LOADER_ID = "IGNITE_MBEAN_APPEND_CLASS_LOADER_ID"; /** + * If property is set to {@code true}, then Ignite will disable MBeans registration. + * This may be helpful if MBeans are not allowed e.g. for security reasons. + * + * Default is {@code false} + */ + public static final String IGNITE_MBEANS_DISABLED = "IGNITE_MBEANS_DISABLED"; + + /** * Property controlling size of buffer holding last exception. Default value of {@code 1000}. */ public static final String IGNITE_EXCEPTION_REGISTRY_MAX_SIZE = "IGNITE_EXCEPTION_REGISTRY_MAX_SIZE"; http://git-wip-us.apache.org/repos/asf/ignite/blob/9fa78849/modules/core/src/main/java/org/apache/ignite/cache/CacheManager.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheManager.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheManager.java index 97409fa..351cd0d 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/CacheManager.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheManager.java @@ -44,6 +44,7 @@ import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.IgnitionEx; import org.apache.ignite.internal.mxbean.IgniteStandardMXBean; import org.apache.ignite.internal.processors.cache.IgniteCacheProxy; +import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; @@ -314,6 +315,9 @@ public class CacheManager implements javax.cache.CacheManager { /** {@inheritDoc} */ @Override public void enableManagement(String cacheName, boolean enabled) { + if(IgniteUtils.IGNITE_MBEANS_DISABLED) + return; + kernalGateway.readLock(); try { @@ -336,6 +340,9 @@ public class CacheManager implements javax.cache.CacheManager { /** {@inheritDoc} */ @Override public void enableStatistics(String cacheName, boolean enabled) { + if(IgniteUtils.IGNITE_MBEANS_DISABLED) + return; + kernalGateway.readLock(); try { @@ -389,6 +396,9 @@ public class CacheManager implements javax.cache.CacheManager { * @param beanType Mxbean name. */ private void unregisterCacheObject(String name, String beanType) { + if(IgniteUtils.IGNITE_MBEANS_DISABLED) + return; + MBeanServer mBeanSrv = ignite.configuration().getMBeanServer(); Set registeredObjNames = mBeanSrv.queryNames(getObjectName(name, beanType), null); http://git-wip-us.apache.org/repos/asf/ignite/blob/9fa78849/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 0c17b32..00c1d73 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 @@ -1605,6 +1605,9 @@ 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(), @@ -1626,6 +1629,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable { /** @throws IgniteCheckedException If registration failed. */ private void registerLocalNodeMBean() throws IgniteCheckedException { + if(U.IGNITE_MBEANS_DISABLED) + return; + ClusterLocalNodeMetricsMXBean mbean = new ClusterLocalNodeMetricsMXBeanImpl(ctx.discovery().localNode()); try { @@ -1662,7 +1668,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable { ExecutorService mgmtExecSvc, ExecutorService restExecSvc, ExecutorService qryExecSvc, - ExecutorService schemaExecSvc) throws IgniteCheckedException { + ExecutorService schemaExecSvc + ) throws IgniteCheckedException {if(U.IGNITE_MBEANS_DISABLED) + return; pubExecSvcMBean = registerExecutorMBean(execSvc, "GridExecutionExecutor"); sysExecSvcMBean = registerExecutorMBean(sysExecSvc, "GridSystemExecutor"); mgmtExecSvcMBean = registerExecutorMBean(mgmtExecSvc, "GridManagementExecutor"); @@ -1684,6 +1692,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable { */ private ObjectName registerExecutorMBean(ExecutorService exec, String name) throws IgniteCheckedException { assert exec != null; + assert !U.IGNITE_MBEANS_DISABLED; try { ObjectName res = U.registerMBean( @@ -1710,24 +1719,26 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable { * @throws IgniteCheckedException If registration failed. */ private void registerStripedExecutorMBean(StripedExecutor stripedExecSvc) throws IgniteCheckedException { - if (stripedExecSvc != null) { - String name = "StripedExecutor"; + if (stripedExecSvc == null || U.IGNITE_MBEANS_DISABLED) + return; - try { - stripedExecSvcMBean = U.registerMBean( - cfg.getMBeanServer(), - cfg.getIgniteInstanceName(), - "Thread Pools", - name, - new StripedExecutorMXBeanAdapter(stripedExecSvc), - StripedExecutorMXBean.class); + String name = "StripedExecutor"; - 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); - } + 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); } } @@ -1738,22 +1749,24 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable { * @return {@code True} if successfully unregistered, {@code false} otherwise. */ private boolean unregisterMBean(@Nullable ObjectName mbean) { - if (mbean != null) - try { - cfg.getMBeanServer().unregisterMBean(mbean); + if (mbean == null) + return true; - if (log.isDebugEnabled()) - log.debug("Unregistered MBean: " + mbean); + assert !U.IGNITE_MBEANS_DISABLED; - return true; - } - catch (JMException e) { - U.error(log, "Failed to unregister MBean.", e); + try { + cfg.getMBeanServer().unregisterMBean(mbean); - return false; - } + if (log.isDebugEnabled()) + log.debug("Unregistered MBean: " + mbean); - return true; + return true; + } + catch (JMException e) { + U.error(log, "Failed to unregister MBean.", e); + + return false; + } } /** http://git-wip-us.apache.org/repos/asf/ignite/blob/9fa78849/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 187d4af..d219333 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 @@ -2120,7 +2120,7 @@ public class IgnitionEx { if (myCfg.getUserAttributes() == null) myCfg.setUserAttributes(Collections.emptyMap()); - if (myCfg.getMBeanServer() == null) + if (myCfg.getMBeanServer() == null && !U.IGNITE_MBEANS_DISABLED) myCfg.setMBeanServer(ManagementFactory.getPlatformMBeanServer()); Marshaller marsh = myCfg.getMarshaller(); @@ -2570,6 +2570,11 @@ public class IgnitionEx { * @throws IgniteCheckedException If registration failed. */ private void registerFactoryMbean(MBeanServer srv) throws IgniteCheckedException { + if(U.IGNITE_MBEANS_DISABLED) + return; + + assert srv != null; + synchronized (mbeans) { GridMBeanServerData data = mbeans.get(srv); @@ -2620,6 +2625,9 @@ public class IgnitionEx { * Unregister delegate Mbean instance for {@link Ignition}. */ private void unregisterFactoryMBean() { + if(U.IGNITE_MBEANS_DISABLED) + return; + synchronized (mbeans) { Iterator> iter = mbeans.entrySet().iterator(); http://git-wip-us.apache.org/repos/asf/ignite/blob/9fa78849/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java index cb8325b..da55ec7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java @@ -169,6 +169,32 @@ public class GridTcpRouterImpl implements GridTcpRouter, GridTcpRouterMBean, Lif "are in use) [firstPort=" + cfg.getPort() + ", lastPort=" + (cfg.getPort() + cfg.getPortRange()) + ", addr=" + hostAddr + ']'); + registerMBean(); + } + + /** + * Stops this router. + */ + @Override public void stop() { + if (srv != null) + srv.stop(); + + if (client != null) + client.stop(true); + + unregisterMBean(); + + if (log.isInfoEnabled()) + log.info("TCP router successfully stopped."); + } + + /** + * Try to register MBean. + */ + private void registerMBean() { + if (U.IGNITE_MBEANS_DISABLED) + return; + try { ObjectName objName = U.registerMBean( ManagementFactory.getPlatformMBeanServer(), @@ -189,28 +215,23 @@ public class GridTcpRouterImpl implements GridTcpRouter, GridTcpRouterMBean, Lif } /** - * Stops this router. + * Unregister MBean. */ - @Override public void stop() { - if (srv != null) - srv.stop(); - - if (client != null) - client.stop(true); + private void unregisterMBean() { + if (mbeanName == null) + return; - if (mbeanName != null) - try { - ManagementFactory.getPlatformMBeanServer().unregisterMBean(mbeanName); + assert !U.IGNITE_MBEANS_DISABLED; - if (log.isDebugEnabled()) - log.debug("Unregistered MBean: " + mbeanName); - } - catch (JMException e) { - U.error(log, "Failed to unregister MBean.", e); - } + try { + ManagementFactory.getPlatformMBeanServer().unregisterMBean(mbeanName); - if (log.isInfoEnabled()) - log.info("TCP router successfully stopped."); + if (log.isDebugEnabled()) + log.debug("Unregistered MBean: " + mbeanName); + } + catch (JMException e) { + U.error(log, "Failed to unregister MBean.", e); + } } /** http://git-wip-us.apache.org/repos/asf/ignite/blob/9fa78849/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index 5149d4b..5b709b3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -37,7 +37,6 @@ import java.util.concurrent.CountDownLatch; import javax.cache.configuration.Factory; import javax.cache.integration.CacheLoader; import javax.cache.integration.CacheWriter; -import javax.management.JMException; import javax.management.MBeanServer; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -3584,6 +3583,9 @@ public class GridCacheProcessor extends GridProcessorAdapter { @SuppressWarnings("unchecked") private void registerMbean(Object obj, @Nullable String cacheName, boolean near) throws IgniteCheckedException { + if(U.IGNITE_MBEANS_DISABLED) + return; + assert obj != null; MBeanServer srvr = ctx.config().getMBeanServer(); @@ -3602,7 +3604,7 @@ public class GridCacheProcessor extends GridProcessorAdapter { U.registerCacheMBean(srvr, ctx.igniteInstanceName(), cacheName, obj.getClass().getName(), mbeanImpl, (Class)itf); } - catch (JMException e) { + catch (Throwable e) { throw new IgniteCheckedException("Failed to register MBean for component: " + obj, e); } @@ -3619,6 +3621,9 @@ public class GridCacheProcessor extends GridProcessorAdapter { * @param near Near flag. */ private void unregisterMbean(Object o, @Nullable String cacheName, boolean near) { + if(U.IGNITE_MBEANS_DISABLED) + return; + assert o != null; MBeanServer srvr = ctx.config().getMBeanServer(); @@ -3645,7 +3650,7 @@ public class GridCacheProcessor extends GridProcessorAdapter { try { srvr.unregisterMBean(U.makeCacheMBeanName(ctx.igniteInstanceName(), cacheName, o.getClass().getName())); } - catch (JMException e) { + catch (Throwable e) { U.error(log, "Failed to unregister MBean for component: " + o, e); } } http://git-wip-us.apache.org/repos/asf/ignite/blob/9fa78849/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java index d147f36..39038ba 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java @@ -477,9 +477,13 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan } /** + * Try to register Metrics MBean. * @throws IgniteCheckedException If failed. */ private void registrateMetricsMBean() throws IgniteCheckedException { + if (U.IGNITE_MBEANS_DISABLED) + return; + try { persistenceMetricsMbeanName = U.registerMBean( cctx.kernalContext().config().getMBeanServer(), @@ -489,25 +493,25 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan persStoreMetrics, PersistenceMetricsMXBean.class); } - catch (JMException e) { + catch (Throwable e) { throw new IgniteCheckedException("Failed to register " + MBEAN_NAME + " MBean.", e); } } /** - * + * Unregister metrics MBean. */ private void unRegistrateMetricsMBean() { - if (persistenceMetricsMbeanName != null) { - try { - cctx.kernalContext().config().getMBeanServer().unregisterMBean(persistenceMetricsMbeanName); - } - catch (InstanceNotFoundException ignore) { - // No-op, nothing to unregister. - } - catch (MBeanRegistrationException e) { - U.error(log, "Failed to unregister " + MBEAN_NAME + " MBean.", e); - } + if (persistenceMetricsMbeanName == null) + return; + + assert !U.IGNITE_MBEANS_DISABLED; + + try { + cctx.kernalContext().config().getMBeanServer().unregisterMBean(persistenceMetricsMbeanName); + } + catch (Throwable e) { + U.error(log, "Failed to unregister " + MBEAN_NAME + " MBean.", e); } } http://git-wip-us.apache.org/repos/asf/ignite/blob/9fa78849/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java index c503fb4..eec3b85 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java @@ -115,6 +115,9 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap * Registers MBeans for all MemoryMetrics configured in this instance. */ private void registerMetricsMBeans() { + if(U.IGNITE_MBEANS_DISABLED) + return; + IgniteConfiguration cfg = cctx.gridConfig(); for (MemoryMetrics memMetrics : memMetricsMap.values()) { @@ -134,6 +137,8 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap MemoryPolicyConfiguration memPlcCfg, IgniteConfiguration cfg ) { + assert !U.IGNITE_MBEANS_DISABLED; + try { U.registerMBean( cfg.getMBeanServer(), @@ -143,7 +148,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap new MemoryMetricsMXBeanImpl(memMetrics, memPlcCfg), MemoryMetricsMXBean.class); } - catch (JMException e) { + catch (Throwable e) { U.error(log, "Failed to register MBean for MemoryMetrics with name: '" + memMetrics.getName() + "'", e); } } @@ -647,19 +652,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap memPlc.evictionTracker().stop(); - IgniteConfiguration cfg = cctx.gridConfig(); - - try { - cfg.getMBeanServer().unregisterMBean( - U.makeMBeanName( - cfg.getIgniteInstanceName(), - "MemoryMetrics", - memPlc.memoryMetrics().getName())); - } - catch (JMException e) { - U.error(log, "Failed to unregister MBean for memory metrics: " + - memPlc.memoryMetrics().getName(), e); - } + unregisterMBean(memPlc.memoryMetrics().getName()); } memPlcMap.clear(); @@ -669,6 +662,29 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap } /** + * Unregister MBean. + * @param name Name of mbean. + */ + private void unregisterMBean(String name) { + if(U.IGNITE_MBEANS_DISABLED) + return; + + IgniteConfiguration cfg = cctx.gridConfig(); + + try { + cfg.getMBeanServer().unregisterMBean( + U.makeMBeanName( + cfg.getIgniteInstanceName(), + "MemoryMetrics", name + )); + } + catch (Throwable e) { + U.error(log, "Failed to unregister MBean for memory metrics: " + + name, e); + } + } + + /** * */ public boolean persistenceEnabled() { http://git-wip-us.apache.org/repos/asf/ignite/blob/9fa78849/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java index 6b94d09..6a3be55 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java @@ -139,6 +139,7 @@ import java.util.zip.ZipInputStream; import java.util.zip.ZipOutputStream; import javax.management.DynamicMBean; import javax.management.JMException; +import javax.management.MBeanRegistrationException; import javax.management.MBeanServer; import javax.management.MalformedObjectNameException; import javax.management.ObjectName; @@ -155,6 +156,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteClientDisconnectedException; import org.apache.ignite.IgniteDeploymentException; import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteIllegalStateException; import org.apache.ignite.IgniteInterruptedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.IgniteSystemProperties; @@ -515,11 +517,14 @@ public abstract class IgniteUtils { } }; + /** Ignite MBeans disabled flag. */ + public static boolean IGNITE_MBEANS_DISABLED = IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_MBEANS_DISABLED); + /** */ private static final boolean assertionsEnabled; /* - * + * Initializes enterprise check. */ static { boolean assertionsEnabled0 = true; @@ -4486,10 +4491,12 @@ public abstract class IgniteUtils { * @param impl MBean implementation. * @param itf MBean interface. * @return JMX object name. + * @throws MBeanRegistrationException if MBeans are disabled. * @throws JMException If MBean creation failed. */ - public static ObjectName registerMBean(MBeanServer mbeanSrv, @Nullable String igniteInstanceName, - @Nullable String grp, String name, T impl, @Nullable Class itf) throws JMException { + public static ObjectName registerMBean(MBeanServer mbeanSrv, @Nullable String igniteInstanceName, @Nullable String grp, + String name, T impl, @Nullable Class itf) throws JMException {if(IGNITE_MBEANS_DISABLED) + throw new MBeanRegistrationException(new IgniteIllegalStateException("No MBeans are allowed.")); assert mbeanSrv != null; assert name != null; assert itf != null; @@ -4510,10 +4517,15 @@ public abstract class IgniteUtils { * @param impl MBean implementation. * @param itf MBean interface. * @return JMX object name. + * @throws MBeanRegistrationException if MBeans are disabled. * @throws JMException If MBean creation failed. + * @throws IgniteException If MBean creation are not allowed. */ public static ObjectName registerMBean(MBeanServer mbeanSrv, ObjectName name, T impl, Class itf) throws JMException { + if(IGNITE_MBEANS_DISABLED) + throw new MBeanRegistrationException(new IgniteIllegalStateException("MBeans are disabled.")); + assert mbeanSrv != null; assert name != null; assert itf != null; @@ -4536,10 +4548,15 @@ public abstract class IgniteUtils { * @param impl MBean implementation. * @param itf MBean interface. * @return JMX object name. + * @throws MBeanRegistrationException if MBeans are disabled. * @throws JMException If MBean creation failed. + * @throws IgniteException If MBean creation are not allowed. */ public static ObjectName registerCacheMBean(MBeanServer mbeanSrv, @Nullable String igniteInstanceName, @Nullable String cacheName, String name, T impl, Class itf) throws JMException { + if(IGNITE_MBEANS_DISABLED) + throw new MBeanRegistrationException(new IgniteIllegalStateException("MBeans are disabled.")); + assert mbeanSrv != null; assert name != null; assert itf != null; http://git-wip-us.apache.org/repos/asf/ignite/blob/9fa78849/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java index 59c2656..07ba214 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java @@ -401,9 +401,11 @@ public abstract class IgniteSpiAdapter implements IgniteSpi { * @param Type of the MBean * @throws IgniteSpiException If registration failed. */ - protected final void registerMBean( - String igniteInstanceName, T impl, Class mbeanItf - ) throws IgniteSpiException { + protected final void registerMBean(String igniteInstanceName, T impl, Class mbeanItf + ) throws IgniteSpiException { + if(ignite == null || U.IGNITE_MBEANS_DISABLED) + return; + MBeanServer jmx = ignite.configuration().getMBeanServer(); assert mbeanItf == null || mbeanItf.isInterface(); @@ -428,6 +430,8 @@ public abstract class IgniteSpiAdapter implements IgniteSpi { protected final void unregisterMBean() throws IgniteSpiException { // Unregister SPI MBean. if (spiMBean != null && ignite != null) { + assert !U.IGNITE_MBEANS_DISABLED; + MBeanServer jmx = ignite.configuration().getMBeanServer(); assert jmx != null; http://git-wip-us.apache.org/repos/asf/ignite/blob/9fa78849/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 1298a75..20f37f7 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 @@ -46,6 +46,7 @@ import org.apache.ignite.util.GridPartitionMapSelfTest; import org.apache.ignite.util.GridQueueSelfTest; import org.apache.ignite.util.GridSpinReadWriteLockSelfTest; import org.apache.ignite.util.GridStringBuilderFactorySelfTest; +import org.apache.ignite.util.mbeans.GridMBeanDisableSelfTest; import org.apache.ignite.util.mbeans.GridMBeanSelfTest; /** @@ -77,6 +78,7 @@ public class IgniteUtilSelfTestSuite extends TestSuite { suite.addTestSuite(GridToStringBuilderSelfTest.class); suite.addTestSuite(GridByteArrayListSelfTest.class); suite.addTestSuite(GridMBeanSelfTest.class); + suite.addTestSuite(GridMBeanDisableSelfTest.class); suite.addTestSuite(GridLongListSelfTest.class); suite.addTestSuite(GridIntListSelfTest.class); suite.addTestSuite(GridArraysSelfTest.class); http://git-wip-us.apache.org/repos/asf/ignite/blob/9fa78849/modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanDisableSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanDisableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanDisableSelfTest.java new file mode 100644 index 0000000..f08f58b --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanDisableSelfTest.java @@ -0,0 +1,121 @@ +/* + * 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.util.mbeans; + +import java.util.concurrent.Callable; +import javax.management.MBeanRegistrationException; +import javax.management.MBeanServer; +import javax.management.ObjectName; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.util.IgniteUtils; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Disabling MBeans test. + */ +public class GridMBeanDisableSelfTest extends GridCommonAbstractTest { + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + IgniteUtils.IGNITE_MBEANS_DISABLED = true; + + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + IgniteUtils.IGNITE_MBEANS_DISABLED = false; + } + + /** + * Test MBean registration. + * + * @throws Exception Thrown if test fails. + */ + public void testCorrectMBeanInfo() throws Exception { + // Node should start and stopped with no errors. + try (final Ignite ignite = startGrid(0)) { + + // Cache should be created and closed with no errors. + try (IgniteCache cache = ignite.getOrCreateCache(new CacheConfiguration("MyCache"))) { + + final MBeanServer server = ignite.configuration().getMBeanServer(); + + GridTestUtils.assertThrowsWithCause( + new Callable() { + @Override public Void call() throws Exception { + U.registerMBean(server, ignite.name(), "dummy", "DummyMbean1", new DummyMBeanImpl(), DummyMBean.class); + + return null; + + } + }, MBeanRegistrationException.class); + + GridTestUtils.assertThrowsWithCause( + new Callable() { + @Override public Void call() throws Exception { + ObjectName objName = U.makeMBeanName( + ignite.name(), + "dummy", + "DummyMbean2" + ); + + U.registerMBean(server, objName, new DummyMBeanImpl(), DummyMBean.class); + + return null; + + } + }, MBeanRegistrationException.class); + + GridTestUtils.assertThrowsWithCause( + new Callable() { + @Override public Void call() throws Exception { + U.registerCacheMBean(server, ignite.name(), "MyCache", "DummyMbean3", + new DummyMBeanImpl(), DummyMBean.class); + + return null; + + } + }, MBeanRegistrationException.class); + } + } + } + + /** + * MBean dummy interface. + */ + interface DummyMBean { + /** */ + void noop(); + } + + /** + * MBean stub. + */ + static class DummyMBeanImpl implements DummyMBean { + /** {@inheritDoc} */ + @Override public void noop() { + // No op. + } + } +} \ No newline at end of file