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 81573200BC2 for ; Thu, 17 Nov 2016 11:14:31 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 801F2160B0B; Thu, 17 Nov 2016 10:14:31 +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 B21AD160AFF for ; Thu, 17 Nov 2016 11:14:29 +0100 (CET) Received: (qmail 13805 invoked by uid 500); 17 Nov 2016 10:14:28 -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 13796 invoked by uid 99); 17 Nov 2016 10:14:28 -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; Thu, 17 Nov 2016 10:14:28 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id C77DBE0C0A; Thu, 17 Nov 2016 10:14:28 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: ptupitsyn@apache.org To: commits@ignite.apache.org Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: ignite git commit: IGNITE-4236 .NET: Add ICache.LocalMetrics and ICache.Metrics(clusterGroup) Date: Thu, 17 Nov 2016 10:14:28 +0000 (UTC) archived-at: Thu, 17 Nov 2016 10:14:31 -0000 Repository: ignite Updated Branches: refs/heads/master 19a62db57 -> 7a15a1330 IGNITE-4236 .NET: Add ICache.LocalMetrics and ICache.Metrics(clusterGroup) This closes #1243 Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/7a15a133 Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/7a15a133 Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/7a15a133 Branch: refs/heads/master Commit: 7a15a1330b1ed37318bdfdb53e4a156842cfa159 Parents: 19a62db Author: Pavel Tupitsyn Authored: Thu Nov 17 13:13:42 2016 +0300 Committer: Pavel Tupitsyn Committed: Thu Nov 17 13:13:42 2016 +0300 ---------------------------------------------------------------------- .../platform/cache/PlatformCache.java | 16 +- .../platform/cluster/PlatformClusterGroup.java | 16 + .../utils/PlatformConfigurationUtils.java | 2 + .../ExpiryCacheHolderTest.cs | 11 + .../Cache/CacheAbstractTest.cs | 18 +- .../Cache/CacheConfigurationTest.cs | 4 +- .../Cache/CacheMetricsTest.cs | 300 +++++++++++++------ .../Cache/CacheSwapSpaceTest.cs | 2 +- .../Cache/CacheTestAsyncWrapper.cs | 13 + .../IgniteConfigurationSerializerTest.cs | 9 +- .../Cache/Configuration/CacheConfiguration.cs | 8 + .../dotnet/Apache.Ignite.Core/Cache/ICache.cs | 19 +- .../IgniteConfigurationSection.xsd | 5 + .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs | 28 +- .../Apache.Ignite.Core/Impl/Cache/CacheOp.cs | 5 +- .../Impl/Cluster/ClusterGroupImpl.cs | 20 ++ 16 files changed, 361 insertions(+), 115 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/7a15a133/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java index 4154c48..aec3703 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java @@ -148,7 +148,7 @@ public class PlatformCache extends PlatformAbstractTarget { public static final int OP_LOCK_ALL = 23; /** */ - public static final int OP_METRICS = 24; + public static final int OP_LOCAL_METRICS = 24; /** */ private static final int OP_PEEK = 25; @@ -324,6 +324,9 @@ public class PlatformCache extends PlatformAbstractTarget { /** */ public static final int OP_EXTENSION = 82; + /** */ + public static final int OP_GLOBAL_METRICS = 83; + /** Underlying JCache in binary mode. */ private final IgniteCacheProxy cache; @@ -907,12 +910,21 @@ public class PlatformCache extends PlatformAbstractTarget { break; - case OP_METRICS: + case OP_LOCAL_METRICS: { CacheMetrics metrics = cache.localMetrics(); writeCacheMetrics(writer, metrics); break; + } + + case OP_GLOBAL_METRICS: { + CacheMetrics metrics = cache.metrics(); + + writeCacheMetrics(writer, metrics); + + break; + } case OP_GET_CONFIG: CacheConfiguration ccfg = ((IgniteCache)cache). http://git-wip-us.apache.org/repos/asf/ignite/blob/7a15a133/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java index 724eea0..dc73468 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java @@ -19,6 +19,8 @@ package org.apache.ignite.internal.processors.platform.cluster; import java.util.Collection; import java.util.UUID; + +import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteCluster; import org.apache.ignite.cluster.ClusterMetrics; @@ -28,6 +30,7 @@ import org.apache.ignite.internal.binary.BinaryRawReaderEx; import org.apache.ignite.internal.binary.BinaryRawWriterEx; import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget; import org.apache.ignite.internal.processors.platform.PlatformContext; +import org.apache.ignite.internal.processors.platform.cache.PlatformCache; import org.apache.ignite.internal.processors.platform.utils.PlatformUtils; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.jetbrains.annotations.Nullable; @@ -97,6 +100,9 @@ public class PlatformClusterGroup extends PlatformAbstractTarget { /** */ private static final int OP_FOR_SERVERS = 23; + /** */ + private static final int OP_CACHE_METRICS = 24; + /** Projection. */ private final ClusterGroupEx prj; @@ -195,6 +201,16 @@ public class PlatformClusterGroup extends PlatformAbstractTarget { break; } + case OP_CACHE_METRICS: { + String cacheName = reader.readString(); + + IgniteCache cache = platformCtx.kernalContext().grid().cache(cacheName); + + PlatformCache.writeCacheMetrics(writer, cache.metrics(prj)); + + break; + } + default: super.processInStreamOutStream(type, reader, writer); } http://git-wip-us.apache.org/repos/asf/ignite/blob/7a15a133/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java index 83cad82..f845675 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java @@ -167,6 +167,7 @@ public class PlatformConfigurationUtils { ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.fromOrdinal(in.readInt())); ccfg.setReadThrough(in.readBoolean()); ccfg.setWriteThrough(in.readBoolean()); + ccfg.setStatisticsEnabled(in.readBoolean()); Object storeFactory = in.readObjectDetached(); @@ -762,6 +763,7 @@ public class PlatformConfigurationUtils { writeEnumInt(writer, ccfg.getWriteSynchronizationMode()); writer.writeBoolean(ccfg.isReadThrough()); writer.writeBoolean(ccfg.isWriteThrough()); + writer.writeBoolean(ccfg.isStatisticsEnabled()); if (ccfg.getCacheStoreFactory() instanceof PlatformDotNetCacheStoreFactoryNative) writer.writeObject(((PlatformDotNetCacheStoreFactoryNative)ccfg.getCacheStoreFactory()).getNativeFactory()); http://git-wip-us.apache.org/repos/asf/ignite/blob/7a15a133/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/ExpiryCacheHolderTest.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/ExpiryCacheHolderTest.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/ExpiryCacheHolderTest.cs index c12fe93..64c9884 100644 --- a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/ExpiryCacheHolderTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/ExpiryCacheHolderTest.cs @@ -29,6 +29,7 @@ namespace Apache.Ignite.AspNet.Tests using Apache.Ignite.Core.Cache.Expiry; using Apache.Ignite.Core.Cache.Query; using Apache.Ignite.Core.Cache.Query.Continuous; + using Apache.Ignite.Core.Cluster; using NUnit.Framework; /// @@ -478,6 +479,16 @@ namespace Apache.Ignite.AspNet.Tests throw new NotImplementedException(); } + public ICacheMetrics GetMetrics(IClusterGroup clusterGroup) + { + throw new NotImplementedException(); + } + + public ICacheMetrics GetLocalMetrics() + { + throw new NotImplementedException(); + } + public Task Rebalance() { throw new NotImplementedException(); http://git-wip-us.apache.org/repos/asf/ignite/blob/7a15a133/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs index 98a2c89..821a179 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs @@ -3152,27 +3152,13 @@ namespace Apache.Ignite.Core.Tests.Cache } [Test] - public void TestCacheMetrics() - { - var cache = Cache(); - - cache.Put(1, 1); - - var m = cache.GetMetrics(); - - Assert.AreEqual(cache.Name, m.CacheName); - - Assert.AreEqual(cache.GetSize(), m.Size); - } - - [Test] public void TestRebalance() { var cache = Cache(); - var fut = cache.Rebalance(); + var task = cache.Rebalance(); - + task.Wait(); } [Test] http://git-wip-us.apache.org/repos/asf/ignite/blob/7a15a133/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs index b02aae4..9d55160 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs @@ -251,6 +251,7 @@ namespace Apache.Ignite.Core.Tests.Cache Assert.AreEqual(x.WriteBehindEnabled, y.WriteBehindEnabled); Assert.AreEqual(x.WriteBehindFlushFrequency, y.WriteBehindFlushFrequency); Assert.AreEqual(x.WriteBehindFlushSize, y.WriteBehindFlushSize); + Assert.AreEqual(x.EnableStatistics, y.EnableStatistics); if (x.ExpiryPolicyFactory != null) Assert.AreEqual(x.ExpiryPolicyFactory.CreateInstance().GetType(), @@ -563,7 +564,8 @@ namespace Apache.Ignite.Core.Tests.Cache Partitions = 513, ExcludeNeighbors = true }, - ExpiryPolicyFactory = new ExpiryFactory() + ExpiryPolicyFactory = new ExpiryFactory(), + EnableStatistics = true }; } /// http://git-wip-us.apache.org/repos/asf/ignite/blob/7a15a133/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheMetricsTest.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheMetricsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheMetricsTest.cs index 0114280..9da037c 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheMetricsTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheMetricsTest.cs @@ -17,7 +17,11 @@ namespace Apache.Ignite.Core.Tests.Cache { + using System; + using System.Threading; using Apache.Ignite.Core.Cache; + using Apache.Ignite.Core.Cache.Configuration; + using Apache.Ignite.Core.Discovery.Tcp; using Apache.Ignite.Core.Impl; using Apache.Ignite.Core.Impl.Cache; using NUnit.Framework; @@ -27,102 +31,222 @@ namespace Apache.Ignite.Core.Tests.Cache /// public class CacheMetricsTest { + /** */ + private const string SecondGridName = "grid"; + + /// + /// Fixture set up. + /// + [TestFixtureSetUp] + public void FixtureSetUp() + { + Ignition.Start(TestUtils.GetTestConfiguration()); + Ignition.Start(new IgniteConfiguration(TestUtils.GetTestConfiguration()) {GridName = SecondGridName }); + } + + /// + /// Fixture tear down. + /// + [TestFixtureTearDown] + public void FixtureTearDown() + { + Ignition.StopAll(true); + } + + /// + /// Tests the local metrics. + /// + [Test] + public void TestLocalMetrics() + { + var metrics = GetLocalRemoteMetrics("localMetrics", c => c.GetLocalMetrics()); + + var localMetrics = metrics.Item1; + var remoteMetrics = metrics.Item2; + + Assert.AreEqual(1, localMetrics.Size); + Assert.AreEqual(1, localMetrics.CacheGets); + Assert.AreEqual(1, localMetrics.CachePuts); + + Assert.AreEqual(0, remoteMetrics.Size); + Assert.AreEqual(0, remoteMetrics.CacheGets); + Assert.AreEqual(0, remoteMetrics.CachePuts); + } + + /// + /// Tests the global metrics. + /// + [Test] + public void TestGlobalMetrics() + { + var metrics = GetLocalRemoteMetrics("globalMetrics", c => c.GetMetrics()); + + var localMetrics = metrics.Item1; + var remoteMetrics = metrics.Item2; + + Assert.AreEqual(1, localMetrics.Size); + Assert.AreEqual(1, localMetrics.CacheGets); + Assert.AreEqual(1, localMetrics.CachePuts); + + Assert.AreEqual(0, remoteMetrics.Size); + Assert.AreEqual(1, remoteMetrics.CacheGets); + Assert.AreEqual(1, remoteMetrics.CachePuts); + } + + /// + /// Tests the cluster group metrics. + /// + [Test] + public void TestClusterGroupMetrics() + { + var cluster = Ignition.GetIgnite().GetCluster(); + + // Get metrics in reverse way, so that first item is for second node and vice versa. + var metrics = GetLocalRemoteMetrics("clusterMetrics", c => c.GetMetrics(cluster.ForRemotes()), + c => c.GetMetrics(cluster.ForLocal())); + + var localMetrics = metrics.Item2; + var remoteMetrics = metrics.Item1; + + Assert.AreEqual(1, localMetrics.Size); + Assert.AreEqual(1, localMetrics.CacheGets); + Assert.AreEqual(1, localMetrics.CachePuts); + + Assert.AreEqual(1, remoteMetrics.Size); + Assert.AreEqual(0, remoteMetrics.CacheGets); + Assert.AreEqual(0, remoteMetrics.CachePuts); + } + /// /// Tests the metrics propagation. /// [Test] public void TestMetricsPropagation() { - using (var ignite = Ignition.Start(TestUtils.GetTestConfiguration())) + var ignite = Ignition.GetIgnite(); + + using (var inStream = IgniteManager.Memory.Allocate().GetStream()) { - using (var inStream = IgniteManager.Memory.Allocate().GetStream()) - { - var result = ignite.GetCompute().ExecuteJavaTask( - "org.apache.ignite.platform.PlatformCacheWriteMetricsTask", inStream.MemoryPointer); - - Assert.IsTrue(result); - - inStream.SynchronizeInput(); - - var reader = ((Ignite)ignite).Marshaller.StartUnmarshal(inStream); - - ICacheMetrics metrics = new CacheMetricsImpl(reader); - - Assert.AreEqual(1, metrics.CacheHits); - Assert.AreEqual(2, metrics.CacheHitPercentage); - Assert.AreEqual(3, metrics.CacheMisses); - Assert.AreEqual(4, metrics.CacheMissPercentage); - Assert.AreEqual(5, metrics.CacheGets); - Assert.AreEqual(6, metrics.CachePuts); - Assert.AreEqual(7, metrics.CacheRemovals); - Assert.AreEqual(8, metrics.CacheEvictions); - Assert.AreEqual(9, metrics.AverageGetTime); - Assert.AreEqual(10, metrics.AveragePutTime); - Assert.AreEqual(11, metrics.AverageRemoveTime); - Assert.AreEqual(12, metrics.AverageTxCommitTime); - Assert.AreEqual(13, metrics.AverageTxRollbackTime); - Assert.AreEqual(14, metrics.CacheTxCommits); - Assert.AreEqual(15, metrics.CacheTxRollbacks); - Assert.AreEqual("myCache", metrics.CacheName); - Assert.AreEqual(16, metrics.OverflowSize); - Assert.AreEqual(17, metrics.OffHeapGets); - Assert.AreEqual(18, metrics.OffHeapPuts); - Assert.AreEqual(19, metrics.OffHeapRemovals); - Assert.AreEqual(20, metrics.OffHeapEvictions); - Assert.AreEqual(21, metrics.OffHeapHits); - Assert.AreEqual(22, metrics.OffHeapHitPercentage); - Assert.AreEqual(23, metrics.OffHeapMisses); - Assert.AreEqual(24, metrics.OffHeapMissPercentage); - Assert.AreEqual(25, metrics.OffHeapEntriesCount); - Assert.AreEqual(26, metrics.OffHeapPrimaryEntriesCount); - Assert.AreEqual(27, metrics.OffHeapBackupEntriesCount); - Assert.AreEqual(28, metrics.OffHeapAllocatedSize); - Assert.AreEqual(29, metrics.OffHeapMaxSize); - Assert.AreEqual(30, metrics.SwapGets); - Assert.AreEqual(31, metrics.SwapPuts); - Assert.AreEqual(32, metrics.SwapRemovals); - Assert.AreEqual(33, metrics.SwapHits); - Assert.AreEqual(34, metrics.SwapMisses); - Assert.AreEqual(35, metrics.SwapEntriesCount); - Assert.AreEqual(36, metrics.SwapSize); - Assert.AreEqual(37, metrics.SwapHitPercentage); - Assert.AreEqual(38, metrics.SwapMissPercentage); - Assert.AreEqual(39, metrics.Size); - Assert.AreEqual(40, metrics.KeySize); - Assert.AreEqual(true, metrics.IsEmpty); - Assert.AreEqual(41, metrics.DhtEvictQueueCurrentSize); - Assert.AreEqual(42, metrics.TxThreadMapSize); - Assert.AreEqual(43, metrics.TxXidMapSize); - Assert.AreEqual(44, metrics.TxCommitQueueSize); - Assert.AreEqual(45, metrics.TxPrepareQueueSize); - Assert.AreEqual(46, metrics.TxStartVersionCountsSize); - Assert.AreEqual(47, metrics.TxCommittedVersionsSize); - Assert.AreEqual(48, metrics.TxRolledbackVersionsSize); - Assert.AreEqual(49, metrics.TxDhtThreadMapSize); - Assert.AreEqual(50, metrics.TxDhtXidMapSize); - Assert.AreEqual(51, metrics.TxDhtCommitQueueSize); - Assert.AreEqual(52, metrics.TxDhtPrepareQueueSize); - Assert.AreEqual(53, metrics.TxDhtStartVersionCountsSize); - Assert.AreEqual(54, metrics.TxDhtCommittedVersionsSize); - Assert.AreEqual(55, metrics.TxDhtRolledbackVersionsSize); - Assert.AreEqual(true, metrics.IsWriteBehindEnabled); - Assert.AreEqual(56, metrics.WriteBehindFlushSize); - Assert.AreEqual(57, metrics.WriteBehindFlushThreadCount); - Assert.AreEqual(58, metrics.WriteBehindFlushFrequency); - Assert.AreEqual(59, metrics.WriteBehindStoreBatchSize); - Assert.AreEqual(60, metrics.WriteBehindTotalCriticalOverflowCount); - Assert.AreEqual(61, metrics.WriteBehindCriticalOverflowCount); - Assert.AreEqual(62, metrics.WriteBehindErrorRetryCount); - Assert.AreEqual(63, metrics.WriteBehindBufferSize); - Assert.AreEqual("foo", metrics.KeyType); - Assert.AreEqual("bar", metrics.ValueType); - Assert.AreEqual(true, metrics.IsStoreByValue); - Assert.AreEqual(true, metrics.IsStatisticsEnabled); - Assert.AreEqual(true, metrics.IsManagementEnabled); - Assert.AreEqual(true, metrics.IsReadThrough); - Assert.AreEqual(true, metrics.IsWriteThrough); - } + var result = ignite.GetCompute().ExecuteJavaTask( + "org.apache.ignite.platform.PlatformCacheWriteMetricsTask", inStream.MemoryPointer); + + Assert.IsTrue(result); + + inStream.SynchronizeInput(); + + var reader = ((Ignite) ignite).Marshaller.StartUnmarshal(inStream); + + ICacheMetrics metrics = new CacheMetricsImpl(reader); + + Assert.AreEqual(1, metrics.CacheHits); + Assert.AreEqual(2, metrics.CacheHitPercentage); + Assert.AreEqual(3, metrics.CacheMisses); + Assert.AreEqual(4, metrics.CacheMissPercentage); + Assert.AreEqual(5, metrics.CacheGets); + Assert.AreEqual(6, metrics.CachePuts); + Assert.AreEqual(7, metrics.CacheRemovals); + Assert.AreEqual(8, metrics.CacheEvictions); + Assert.AreEqual(9, metrics.AverageGetTime); + Assert.AreEqual(10, metrics.AveragePutTime); + Assert.AreEqual(11, metrics.AverageRemoveTime); + Assert.AreEqual(12, metrics.AverageTxCommitTime); + Assert.AreEqual(13, metrics.AverageTxRollbackTime); + Assert.AreEqual(14, metrics.CacheTxCommits); + Assert.AreEqual(15, metrics.CacheTxRollbacks); + Assert.AreEqual("myCache", metrics.CacheName); + Assert.AreEqual(16, metrics.OverflowSize); + Assert.AreEqual(17, metrics.OffHeapGets); + Assert.AreEqual(18, metrics.OffHeapPuts); + Assert.AreEqual(19, metrics.OffHeapRemovals); + Assert.AreEqual(20, metrics.OffHeapEvictions); + Assert.AreEqual(21, metrics.OffHeapHits); + Assert.AreEqual(22, metrics.OffHeapHitPercentage); + Assert.AreEqual(23, metrics.OffHeapMisses); + Assert.AreEqual(24, metrics.OffHeapMissPercentage); + Assert.AreEqual(25, metrics.OffHeapEntriesCount); + Assert.AreEqual(26, metrics.OffHeapPrimaryEntriesCount); + Assert.AreEqual(27, metrics.OffHeapBackupEntriesCount); + Assert.AreEqual(28, metrics.OffHeapAllocatedSize); + Assert.AreEqual(29, metrics.OffHeapMaxSize); + Assert.AreEqual(30, metrics.SwapGets); + Assert.AreEqual(31, metrics.SwapPuts); + Assert.AreEqual(32, metrics.SwapRemovals); + Assert.AreEqual(33, metrics.SwapHits); + Assert.AreEqual(34, metrics.SwapMisses); + Assert.AreEqual(35, metrics.SwapEntriesCount); + Assert.AreEqual(36, metrics.SwapSize); + Assert.AreEqual(37, metrics.SwapHitPercentage); + Assert.AreEqual(38, metrics.SwapMissPercentage); + Assert.AreEqual(39, metrics.Size); + Assert.AreEqual(40, metrics.KeySize); + Assert.AreEqual(true, metrics.IsEmpty); + Assert.AreEqual(41, metrics.DhtEvictQueueCurrentSize); + Assert.AreEqual(42, metrics.TxThreadMapSize); + Assert.AreEqual(43, metrics.TxXidMapSize); + Assert.AreEqual(44, metrics.TxCommitQueueSize); + Assert.AreEqual(45, metrics.TxPrepareQueueSize); + Assert.AreEqual(46, metrics.TxStartVersionCountsSize); + Assert.AreEqual(47, metrics.TxCommittedVersionsSize); + Assert.AreEqual(48, metrics.TxRolledbackVersionsSize); + Assert.AreEqual(49, metrics.TxDhtThreadMapSize); + Assert.AreEqual(50, metrics.TxDhtXidMapSize); + Assert.AreEqual(51, metrics.TxDhtCommitQueueSize); + Assert.AreEqual(52, metrics.TxDhtPrepareQueueSize); + Assert.AreEqual(53, metrics.TxDhtStartVersionCountsSize); + Assert.AreEqual(54, metrics.TxDhtCommittedVersionsSize); + Assert.AreEqual(55, metrics.TxDhtRolledbackVersionsSize); + Assert.AreEqual(true, metrics.IsWriteBehindEnabled); + Assert.AreEqual(56, metrics.WriteBehindFlushSize); + Assert.AreEqual(57, metrics.WriteBehindFlushThreadCount); + Assert.AreEqual(58, metrics.WriteBehindFlushFrequency); + Assert.AreEqual(59, metrics.WriteBehindStoreBatchSize); + Assert.AreEqual(60, metrics.WriteBehindTotalCriticalOverflowCount); + Assert.AreEqual(61, metrics.WriteBehindCriticalOverflowCount); + Assert.AreEqual(62, metrics.WriteBehindErrorRetryCount); + Assert.AreEqual(63, metrics.WriteBehindBufferSize); + Assert.AreEqual("foo", metrics.KeyType); + Assert.AreEqual("bar", metrics.ValueType); + Assert.AreEqual(true, metrics.IsStoreByValue); + Assert.AreEqual(true, metrics.IsStatisticsEnabled); + Assert.AreEqual(true, metrics.IsManagementEnabled); + Assert.AreEqual(true, metrics.IsReadThrough); + Assert.AreEqual(true, metrics.IsWriteThrough); } } + + /// + /// Creates a cache, performs put-get, returns metrics from both nodes. + /// + private static Tuple GetLocalRemoteMetrics(string cacheName, + Func, ICacheMetrics> func, Func, ICacheMetrics> func2 = null) + { + func2 = func2 ?? func; + + var localCache = Ignition.GetIgnite().CreateCache(new CacheConfiguration(cacheName) + { + EnableStatistics = true + }); + + var remoteCache = Ignition.GetIgnite(SecondGridName).GetCache(cacheName); + + Assert.IsTrue(localCache.GetConfiguration().EnableStatistics); + Assert.IsTrue(remoteCache.GetConfiguration().EnableStatistics); + + localCache.Put(1, 1); + localCache.Get(1); + + // Wait for metrics to propagate. + Thread.Sleep(TcpDiscoverySpi.DefaultHeartbeatFrequency); + + var localMetrics = func(localCache); + Assert.IsTrue(localMetrics.IsStatisticsEnabled); + Assert.AreEqual(cacheName, localMetrics.CacheName); + + var remoteMetrics = func2(remoteCache); + Assert.IsTrue(remoteMetrics.IsStatisticsEnabled); + Assert.AreEqual(cacheName, remoteMetrics.CacheName); + + return Tuple.Create(localMetrics, remoteMetrics); + } } } http://git-wip-us.apache.org/repos/asf/ignite/blob/7a15a133/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheSwapSpaceTest.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheSwapSpaceTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheSwapSpaceTest.cs index f32de21..7b93815 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheSwapSpaceTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheSwapSpaceTest.cs @@ -110,7 +110,7 @@ namespace Apache.Ignite.Core.Tests.Cache // Wait for metrics update and check metrics. Thread.Sleep(((TcpDiscoverySpi) ignite.GetConfiguration().DiscoverySpi).HeartbeatFrequency); - var metrics = cache.GetMetrics(); + var metrics = cache.GetLocalMetrics(); Assert.AreEqual(4, metrics.OffHeapEntriesCount); // Entry takes more space than the value Assert.AreEqual(3, metrics.SwapEntriesCount); // 10 - 3 - 4 = 3 http://git-wip-us.apache.org/repos/asf/ignite/blob/7a15a133/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheTestAsyncWrapper.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheTestAsyncWrapper.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheTestAsyncWrapper.cs index ff0c37c..14dfbca 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheTestAsyncWrapper.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheTestAsyncWrapper.cs @@ -27,6 +27,7 @@ namespace Apache.Ignite.Core.Tests.Cache using Apache.Ignite.Core.Cache.Expiry; using Apache.Ignite.Core.Cache.Query; using Apache.Ignite.Core.Cache.Query.Continuous; + using Apache.Ignite.Core.Cluster; /// /// Wraps IGridCache implementation to simplify async mode testing. @@ -523,6 +524,18 @@ namespace Apache.Ignite.Core.Tests.Cache } /** */ + public ICacheMetrics GetMetrics(IClusterGroup clusterGroup) + { + return _cache.GetMetrics(clusterGroup); + } + + /** */ + public ICacheMetrics GetLocalMetrics() + { + return _cache.GetLocalMetrics(); + } + + /** */ public Task Rebalance() { return _cache.Rebalance(); http://git-wip-us.apache.org/repos/asf/ignite/blob/7a15a133/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs index 4584530..55b8dcf 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs @@ -23,6 +23,7 @@ namespace Apache.Ignite.Core.Tests using System.Collections; using System.Collections.Generic; using System.Configuration; + using System.Diagnostics.CodeAnalysis; using System.Globalization; using System.IO; using System.Linq; @@ -81,7 +82,7 @@ namespace Apache.Ignite.Core.Tests - + @@ -151,6 +152,7 @@ namespace Apache.Ignite.Core.Tests Assert.IsTrue(cacheCfg.ReadThrough); Assert.IsTrue(cacheCfg.WriteThrough); Assert.IsInstanceOf(cacheCfg.ExpiryPolicyFactory); + Assert.IsTrue(cacheCfg.EnableStatistics); var queryEntity = cacheCfg.QueryEntities.Single(); Assert.AreEqual(typeof(int), queryEntity.KeyType); @@ -234,9 +236,9 @@ namespace Apache.Ignite.Core.Tests /// Tests that all properties are present in the schema. /// [Test] + [SuppressMessage("ReSharper", "PossibleNullReferenceException")] public void TestAllPropertiesArePresentInSchema() { - // ReSharper disable once PossibleNullReferenceException var schema = XDocument.Load("IgniteConfigurationSection.xsd") .Root.Elements() .Single(x => x.Attribute("name").Value == "igniteConfiguration"); @@ -639,7 +641,8 @@ namespace Apache.Ignite.Core.Tests ExcludeNeighbors = true, Partitions = 48 }, - ExpiryPolicyFactory = new MyPolicyFactory() + ExpiryPolicyFactory = new MyPolicyFactory(), + EnableStatistics = true } }, ClientMode = true, http://git-wip-us.apache.org/repos/asf/ignite/blob/7a15a133/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs index 3436cd2..2795111 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs @@ -272,6 +272,7 @@ namespace Apache.Ignite.Core.Cache.Configuration WriteSynchronizationMode = (CacheWriteSynchronizationMode) reader.ReadInt(); ReadThrough = reader.ReadBoolean(); WriteThrough = reader.ReadBoolean(); + EnableStatistics = reader.ReadBoolean(); CacheStoreFactory = reader.ReadObject>(); var count = reader.ReadInt(); @@ -328,6 +329,7 @@ namespace Apache.Ignite.Core.Cache.Configuration writer.WriteInt((int) WriteSynchronizationMode); writer.WriteBoolean(ReadThrough); writer.WriteBoolean(WriteThrough); + writer.WriteBoolean(EnableStatistics); writer.WriteObject(CacheStoreFactory); if (QueryEntities != null) @@ -704,5 +706,11 @@ namespace Apache.Ignite.Core.Cache.Configuration /// Default is null, which means no expiration. /// public IFactory ExpiryPolicyFactory { get; set; } + + /// + /// Gets or sets a value indicating whether statistics gathering is enabled on a cache. + /// These statistics can be retrieved via . + /// + public bool EnableStatistics { get; set; } } } http://git-wip-us.apache.org/repos/asf/ignite/blob/7a15a133/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICache.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICache.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICache.cs index 9d72cfa..2a0ec86 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICache.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICache.cs @@ -27,6 +27,7 @@ namespace Apache.Ignite.Core.Cache using Apache.Ignite.Core.Cache.Query; using Apache.Ignite.Core.Cache.Query.Continuous; using Apache.Ignite.Core.Cache.Store; + using Apache.Ignite.Core.Cluster; using Apache.Ignite.Core.Transactions; /// @@ -847,7 +848,7 @@ namespace Apache.Ignite.Core.Cache bool IsLocalLocked(TK key, bool byCurrentThread); /// - /// Gets snapshot metrics (statistics) for this cache. + /// Gets global (whole cluster) snapshot metrics (statistics) for this cache. /// /// Cache metrics. [SuppressMessage("Microsoft.Design", "CA1024:UsePropertiesWhereAppropriate", @@ -855,6 +856,22 @@ namespace Apache.Ignite.Core.Cache ICacheMetrics GetMetrics(); /// + /// Gets global (whole cluster group) snapshot metrics (statistics) for this cache. + /// + /// The cluster group to get metrics for. + [SuppressMessage("Microsoft.Design", "CA1024:UsePropertiesWhereAppropriate", + Justification = "Expensive operation.")] + ICacheMetrics GetMetrics(IClusterGroup clusterGroup); + + /// + /// Gets local snapshot metrics (statistics) for this cache. + /// + /// Cache metrics. + [SuppressMessage("Microsoft.Design", "CA1024:UsePropertiesWhereAppropriate", + Justification = "Expensive operation.")] + ICacheMetrics GetLocalMetrics(); + + /// /// Rebalances cache partitions. This method is usually used when rebalanceDelay configuration parameter /// has non-zero value. When many nodes are started or stopped almost concurrently, /// it is more efficient to delay rebalancing until the node topology is stable to make sure that no redundant http://git-wip-us.apache.org/repos/asf/ignite/blob/7a15a133/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd index 81b2298..b35527d 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd +++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd @@ -718,6 +718,11 @@ Frequency with which write-behind cache is flushed to the cache store. This value defines the maximum time interval between object insertion/deletion from the cache at the moment when corresponding operation is applied to the cache store. + + + Value indicating whether statistics gathering is enabled on a cache. These statistics can be retrieved via ICache.GetMetrics(). + + http://git-wip-us.apache.org/repos/asf/ignite/blob/7a15a133/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs index 556b8bb..e0d1a3c 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs @@ -28,11 +28,13 @@ namespace Apache.Ignite.Core.Impl.Cache using Apache.Ignite.Core.Cache.Expiry; using Apache.Ignite.Core.Cache.Query; using Apache.Ignite.Core.Cache.Query.Continuous; + using Apache.Ignite.Core.Cluster; using Apache.Ignite.Core.Impl.Binary; using Apache.Ignite.Core.Impl.Binary.IO; using Apache.Ignite.Core.Impl.Cache.Expiry; using Apache.Ignite.Core.Impl.Cache.Query; using Apache.Ignite.Core.Impl.Cache.Query.Continuous; + using Apache.Ignite.Core.Impl.Cluster; using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Core.Impl.Unmanaged; @@ -894,7 +896,31 @@ namespace Apache.Ignite.Core.Impl.Cache /** */ public ICacheMetrics GetMetrics() { - return DoInOp((int)CacheOp.Metrics, stream => + return DoInOp((int) CacheOp.GlobalMetrics, stream => + { + IBinaryRawReader reader = Marshaller.StartUnmarshal(stream, false); + + return new CacheMetricsImpl(reader); + }); + } + + /** */ + public ICacheMetrics GetMetrics(IClusterGroup clusterGroup) + { + IgniteArgumentCheck.NotNull(clusterGroup, "clusterGroup"); + + var prj = clusterGroup as ClusterGroupImpl; + + if (prj == null) + throw new ArgumentException("Unexpected IClusterGroup implementation: " + clusterGroup.GetType()); + + return prj.GetCacheMetrics(Name); + } + + /** */ + public ICacheMetrics GetLocalMetrics() + { + return DoInOp((int) CacheOp.LocalMetrics, stream => { IBinaryRawReader reader = Marshaller.StartUnmarshal(stream, false); http://git-wip-us.apache.org/repos/asf/ignite/blob/7a15a133/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs index 8bf3945..dc4f9aa 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs @@ -45,7 +45,7 @@ namespace Apache.Ignite.Core.Impl.Cache LocalClearAll = 21, Lock = 22, LockAll = 23, - Metrics = 24, + LocalMetrics = 24, Peek = 25, Put = 26, PutAll = 27, @@ -103,6 +103,7 @@ namespace Apache.Ignite.Core.Impl.Cache InvokeAsync = 79, InvokeAllAsync = 80, PutIfAbsentAsync = 81, - Extension = 82 + Extension = 82, + GlobalMetrics = 83 } } \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/7a15a133/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs index 6c8779c..641b6b4 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs @@ -24,11 +24,13 @@ namespace Apache.Ignite.Core.Impl.Cluster using System.Linq; using System.Threading; using Apache.Ignite.Core.Binary; + using Apache.Ignite.Core.Cache; using Apache.Ignite.Core.Cluster; using Apache.Ignite.Core.Common; using Apache.Ignite.Core.Compute; using Apache.Ignite.Core.Events; using Apache.Ignite.Core.Impl.Binary; + using Apache.Ignite.Core.Impl.Cache; using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Core.Impl.Compute; using Apache.Ignite.Core.Impl.Events; @@ -110,6 +112,9 @@ namespace Apache.Ignite.Core.Impl.Cluster /** */ private const int OpForServers = 23; + /** */ + private const int OpCacheMetrics = 24; + /** Initial Ignite instance. */ private readonly Ignite _ignite; @@ -506,6 +511,21 @@ namespace Apache.Ignite.Core.Impl.Cluster } /// + /// Gets the cache metrics within this cluster group. + /// + /// Name of the cache. + /// Metrics. + public ICacheMetrics GetCacheMetrics(string cacheName) + { + return DoOutInOp(OpCacheMetrics, w => w.WriteString(cacheName), stream => + { + IBinaryRawReader reader = Marshaller.StartUnmarshal(stream, false); + + return new CacheMetricsImpl(reader); + }); + } + + /// /// Creates new Cluster Group from given native projection. /// /// Native projection.