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 3BF9D200BC1 for ; Wed, 16 Nov 2016 16:16:34 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 3A6CA160B08; Wed, 16 Nov 2016 15:16:34 +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 12A17160B03 for ; Wed, 16 Nov 2016 16:16:31 +0100 (CET) Received: (qmail 75212 invoked by uid 500); 16 Nov 2016 15:16:31 -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 75202 invoked by uid 99); 16 Nov 2016 15:16:31 -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; Wed, 16 Nov 2016 15:16:31 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 27667E00C7; Wed, 16 Nov 2016 15:16:31 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 8bit From: ptupitsyn@apache.org To: commits@ignite.apache.org Message-Id: <351fa0a7251140f69b7f639691c535e1@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: ignite git commit: IGNITE-4234 .NET: Propagate missing CacheMetrics properties Date: Wed, 16 Nov 2016 15:16:31 +0000 (UTC) archived-at: Wed, 16 Nov 2016 15:16:34 -0000 Repository: ignite Updated Branches: refs/heads/master f1e6257c5 -> 17316f934 IGNITE-4234 .NET: Propagate missing CacheMetrics properties This closes #1239 Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/17316f93 Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/17316f93 Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/17316f93 Branch: refs/heads/master Commit: 17316f934029890169ba17cb05386ffe675775cb Parents: f1e6257 Author: Pavel Tupitsyn Authored: Wed Nov 16 18:15:57 2016 +0300 Committer: Pavel Tupitsyn Committed: Wed Nov 16 18:15:57 2016 +0300 ---------------------------------------------------------------------- .../platform/cache/PlatformCache.java | 144 +++-- .../platform/PlatformCacheWriteMetricsTask.java | 463 ++++++++++++++ .../Apache.Ignite.Core.Tests.csproj | 1 + .../Cache/CacheMetricsTest.cs | 128 ++++ .../Cache/CacheSwapSpaceTest.cs | 4 +- .../Apache.Ignite.Core/Cache/ICacheMetrics.cs | 236 +++++-- .../Impl/Cache/CacheMetricsImpl.cs | 617 ++++++++++++++----- 7 files changed, 1339 insertions(+), 254 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/17316f93/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 677971c..4154c48 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 @@ -21,6 +21,7 @@ import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.binary.BinaryRawReader; +import org.apache.ignite.binary.BinaryRawWriter; import org.apache.ignite.cache.CacheEntryProcessor; import org.apache.ignite.cache.CacheMetrics; import org.apache.ignite.cache.CachePartialUpdateException; @@ -64,8 +65,6 @@ import org.apache.ignite.transactions.TransactionTimeoutException; import org.jetbrains.annotations.Nullable; import javax.cache.Cache; -import javax.cache.expiry.Duration; -import javax.cache.expiry.ExpiryPolicy; import javax.cache.integration.CompletionListener; import javax.cache.processor.EntryProcessorException; import javax.cache.processor.EntryProcessorResult; @@ -80,7 +79,7 @@ import java.util.concurrent.locks.Lock; /** * Native cache wrapper implementation. */ -@SuppressWarnings({"unchecked", "UnusedDeclaration", "TryFinallyCanBeTryWithResources", "TypeMayBeWeakened"}) +@SuppressWarnings({"unchecked", "UnusedDeclaration", "TryFinallyCanBeTryWithResources", "TypeMayBeWeakened", "WeakerAccess"}) public class PlatformCache extends PlatformAbstractTarget { /** */ public static final int OP_CLEAR = 1; @@ -911,59 +910,7 @@ public class PlatformCache extends PlatformAbstractTarget { case OP_METRICS: CacheMetrics metrics = cache.localMetrics(); - writer.writeLong(metrics.getCacheGets()); - writer.writeLong(metrics.getCachePuts()); - writer.writeLong(metrics.getCacheHits()); - writer.writeLong(metrics.getCacheMisses()); - writer.writeLong(metrics.getCacheTxCommits()); - writer.writeLong(metrics.getCacheTxRollbacks()); - writer.writeLong(metrics.getCacheEvictions()); - writer.writeLong(metrics.getCacheRemovals()); - writer.writeFloat(metrics.getAveragePutTime()); - writer.writeFloat(metrics.getAverageGetTime()); - writer.writeFloat(metrics.getAverageRemoveTime()); - writer.writeFloat(metrics.getAverageTxCommitTime()); - writer.writeFloat(metrics.getAverageTxRollbackTime()); - writer.writeString(metrics.name()); - writer.writeLong(metrics.getOverflowSize()); - writer.writeLong(metrics.getOffHeapEntriesCount()); - writer.writeLong(metrics.getOffHeapAllocatedSize()); - writer.writeInt(metrics.getSize()); - writer.writeInt(metrics.getKeySize()); - writer.writeBoolean(metrics.isEmpty()); - writer.writeInt(metrics.getDhtEvictQueueCurrentSize()); - writer.writeInt(metrics.getTxThreadMapSize()); - writer.writeInt(metrics.getTxXidMapSize()); - writer.writeInt(metrics.getTxCommitQueueSize()); - writer.writeInt(metrics.getTxPrepareQueueSize()); - writer.writeInt(metrics.getTxStartVersionCountsSize()); - writer.writeInt(metrics.getTxCommittedVersionsSize()); - writer.writeInt(metrics.getTxRolledbackVersionsSize()); - writer.writeInt(metrics.getTxDhtThreadMapSize()); - writer.writeInt(metrics.getTxDhtXidMapSize()); - writer.writeInt(metrics.getTxDhtCommitQueueSize()); - writer.writeInt(metrics.getTxDhtPrepareQueueSize()); - writer.writeInt(metrics.getTxDhtStartVersionCountsSize()); - writer.writeInt(metrics.getTxDhtCommittedVersionsSize()); - writer.writeInt(metrics.getTxDhtRolledbackVersionsSize()); - writer.writeBoolean(metrics.isWriteBehindEnabled()); - writer.writeInt(metrics.getWriteBehindFlushSize()); - writer.writeInt(metrics.getWriteBehindFlushThreadCount()); - writer.writeLong(metrics.getWriteBehindFlushFrequency()); - writer.writeInt(metrics.getWriteBehindStoreBatchSize()); - writer.writeInt(metrics.getWriteBehindTotalCriticalOverflowCount()); - writer.writeInt(metrics.getWriteBehindCriticalOverflowCount()); - writer.writeInt(metrics.getWriteBehindErrorRetryCount()); - writer.writeInt(metrics.getWriteBehindBufferSize()); - writer.writeString(metrics.getKeyType()); - writer.writeString(metrics.getValueType()); - writer.writeBoolean(metrics.isStoreByValue()); - writer.writeBoolean(metrics.isStatisticsEnabled()); - writer.writeBoolean(metrics.isManagementEnabled()); - writer.writeBoolean(metrics.isReadThrough()); - writer.writeBoolean(metrics.isWriteThrough()); - writer.writeFloat(metrics.getCacheHitPercentage()); - writer.writeFloat(metrics.getCacheMissPercentage()); + writeCacheMetrics(writer, metrics); break; @@ -1362,6 +1309,91 @@ public class PlatformCache extends PlatformAbstractTarget { } /** + * Writes cache metrics. + * + * @param writer Writer. + * @param metrics Metrics. + */ + public static void writeCacheMetrics(BinaryRawWriter writer, CacheMetrics metrics) { + assert writer != null; + assert metrics != null; + + writer.writeLong(metrics.getCacheHits()); + writer.writeFloat(metrics.getCacheHitPercentage()); + writer.writeLong(metrics.getCacheMisses()); + writer.writeFloat(metrics.getCacheMissPercentage()); + writer.writeLong(metrics.getCacheGets()); + writer.writeLong(metrics.getCachePuts()); + writer.writeLong(metrics.getCacheRemovals()); + writer.writeLong(metrics.getCacheEvictions()); + writer.writeFloat(metrics.getAverageGetTime()); + writer.writeFloat(metrics.getAveragePutTime()); + writer.writeFloat(metrics.getAverageRemoveTime()); + writer.writeFloat(metrics.getAverageTxCommitTime()); + writer.writeFloat(metrics.getAverageTxRollbackTime()); + writer.writeLong(metrics.getCacheTxCommits()); + writer.writeLong(metrics.getCacheTxRollbacks()); + writer.writeString(metrics.name()); + writer.writeLong(metrics.getOverflowSize()); + writer.writeLong(metrics.getOffHeapGets()); + writer.writeLong(metrics.getOffHeapPuts()); + writer.writeLong(metrics.getOffHeapRemovals()); + writer.writeLong(metrics.getOffHeapEvictions()); + writer.writeLong(metrics.getOffHeapHits()); + writer.writeFloat(metrics.getOffHeapHitPercentage()); + writer.writeLong(metrics.getOffHeapMisses()); + writer.writeFloat(metrics.getOffHeapMissPercentage()); + writer.writeLong(metrics.getOffHeapEntriesCount()); + writer.writeLong(metrics.getOffHeapPrimaryEntriesCount()); + writer.writeLong(metrics.getOffHeapBackupEntriesCount()); + writer.writeLong(metrics.getOffHeapAllocatedSize()); + writer.writeLong(metrics.getOffHeapMaxSize()); + writer.writeLong(metrics.getSwapGets()); + writer.writeLong(metrics.getSwapPuts()); + writer.writeLong(metrics.getSwapRemovals()); + writer.writeLong(metrics.getSwapHits()); + writer.writeLong(metrics.getSwapMisses()); + writer.writeLong(metrics.getSwapEntriesCount()); + writer.writeLong(metrics.getSwapSize()); + writer.writeFloat(metrics.getSwapHitPercentage()); + writer.writeFloat(metrics.getSwapMissPercentage()); + writer.writeInt(metrics.getSize()); + writer.writeInt(metrics.getKeySize()); + writer.writeBoolean(metrics.isEmpty()); + writer.writeInt(metrics.getDhtEvictQueueCurrentSize()); + writer.writeInt(metrics.getTxThreadMapSize()); + writer.writeInt(metrics.getTxXidMapSize()); + writer.writeInt(metrics.getTxCommitQueueSize()); + writer.writeInt(metrics.getTxPrepareQueueSize()); + writer.writeInt(metrics.getTxStartVersionCountsSize()); + writer.writeInt(metrics.getTxCommittedVersionsSize()); + writer.writeInt(metrics.getTxRolledbackVersionsSize()); + writer.writeInt(metrics.getTxDhtThreadMapSize()); + writer.writeInt(metrics.getTxDhtXidMapSize()); + writer.writeInt(metrics.getTxDhtCommitQueueSize()); + writer.writeInt(metrics.getTxDhtPrepareQueueSize()); + writer.writeInt(metrics.getTxDhtStartVersionCountsSize()); + writer.writeInt(metrics.getTxDhtCommittedVersionsSize()); + writer.writeInt(metrics.getTxDhtRolledbackVersionsSize()); + writer.writeBoolean(metrics.isWriteBehindEnabled()); + writer.writeInt(metrics.getWriteBehindFlushSize()); + writer.writeInt(metrics.getWriteBehindFlushThreadCount()); + writer.writeLong(metrics.getWriteBehindFlushFrequency()); + writer.writeInt(metrics.getWriteBehindStoreBatchSize()); + writer.writeInt(metrics.getWriteBehindTotalCriticalOverflowCount()); + writer.writeInt(metrics.getWriteBehindCriticalOverflowCount()); + writer.writeInt(metrics.getWriteBehindErrorRetryCount()); + writer.writeInt(metrics.getWriteBehindBufferSize()); + writer.writeString(metrics.getKeyType()); + writer.writeString(metrics.getValueType()); + writer.writeBoolean(metrics.isStoreByValue()); + writer.writeBoolean(metrics.isStatisticsEnabled()); + writer.writeBoolean(metrics.isManagementEnabled()); + writer.writeBoolean(metrics.isReadThrough()); + writer.writeBoolean(metrics.isWriteThrough()); + } + + /** * Writes error with EntryProcessorException cause. */ private static class GetAllWriter implements PlatformFutureUtils.Writer { http://git-wip-us.apache.org/repos/asf/ignite/blob/17316f93/modules/core/src/test/java/org/apache/ignite/platform/PlatformCacheWriteMetricsTask.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/platform/PlatformCacheWriteMetricsTask.java b/modules/core/src/test/java/org/apache/ignite/platform/PlatformCacheWriteMetricsTask.java new file mode 100644 index 0000000..593f26d --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/platform/PlatformCacheWriteMetricsTask.java @@ -0,0 +1,463 @@ +/* + * 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.platform; + +import org.apache.ignite.Ignite; +import org.apache.ignite.cache.CacheMetrics; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.compute.ComputeJob; +import org.apache.ignite.compute.ComputeJobAdapter; +import org.apache.ignite.compute.ComputeJobResult; +import org.apache.ignite.compute.ComputeTaskAdapter; +import org.apache.ignite.internal.binary.BinaryRawWriterEx; +import org.apache.ignite.internal.processors.platform.PlatformContext; +import org.apache.ignite.internal.processors.platform.cache.PlatformCache; +import org.apache.ignite.internal.processors.platform.memory.PlatformMemory; +import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream; +import org.apache.ignite.internal.processors.platform.utils.PlatformUtils; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.resources.IgniteInstanceResource; +import org.jetbrains.annotations.Nullable; + +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * Test task writing predefined metrics values to a stream. + */ +@SuppressWarnings("UnusedDeclaration") +public class PlatformCacheWriteMetricsTask extends ComputeTaskAdapter { + /** {@inheritDoc} */ + @Nullable @Override public Map map(List subgrid, Long ptr) { + return Collections.singletonMap(new Job(ptr), F.first(subgrid)); + } + + /** {@inheritDoc} */ + @Nullable @Override public Object reduce(List results) { + return results.get(0).getData(); + } + + /** + * Job. + */ + private static class Job extends ComputeJobAdapter { + /** Grid. */ + @IgniteInstanceResource + protected transient Ignite ignite; + + /** Stream ptr. */ + private final long ptr; + + /** + * Constructor. + * + * @param ptr Stream ptr. + */ + private Job(long ptr) { + this.ptr = ptr; + } + + /** {@inheritDoc} */ + @Nullable @Override public Object execute() { + PlatformContext ctx = PlatformUtils.platformContext(ignite); + + try (PlatformMemory mem = ctx.memory().get(ptr)) { + PlatformOutputStream out = mem.output(); + BinaryRawWriterEx writer = ctx.writer(out); + + PlatformCache.writeCacheMetrics(writer, new TestCacheMetrics()); + + out.synchronize(); + } + + return true; + } + } + + /** + * Predefined metrics. + */ + private static class TestCacheMetrics implements CacheMetrics { + /** {@inheritDoc} */ + @Override public long getCacheHits() { + return 1; + } + + /** {@inheritDoc} */ + @Override public float getCacheHitPercentage() { + return 2; + } + + /** {@inheritDoc} */ + @Override public long getCacheMisses() { + return 3; + } + + /** {@inheritDoc} */ + @Override public float getCacheMissPercentage() { + return 4; + } + + /** {@inheritDoc} */ + @Override public long getCacheGets() { + return 5; + } + + /** {@inheritDoc} */ + @Override public long getCachePuts() { + return 6; + } + + /** {@inheritDoc} */ + @Override public long getCacheRemovals() { + return 7; + } + + /** {@inheritDoc} */ + @Override public long getCacheEvictions() { + return 8; + } + + /** {@inheritDoc} */ + @Override public float getAverageGetTime() { + return 9; + } + + /** {@inheritDoc} */ + @Override public float getAveragePutTime() { + return 10; + } + + /** {@inheritDoc} */ + @Override public float getAverageRemoveTime() { + return 11; + } + + /** {@inheritDoc} */ + @Override public float getAverageTxCommitTime() { + return 12; + } + + /** {@inheritDoc} */ + @Override public float getAverageTxRollbackTime() { + return 13; + } + + /** {@inheritDoc} */ + @Override public long getCacheTxCommits() { + return 14; + } + + /** {@inheritDoc} */ + @Override public long getCacheTxRollbacks() { + return 15; + } + + /** {@inheritDoc} */ + @Override public String name() { + return "myCache"; + } + + /** {@inheritDoc} */ + @Override public long getOverflowSize() { + return 16; + } + + /** {@inheritDoc} */ + @Override public long getOffHeapGets() { + return 17; + } + + /** {@inheritDoc} */ + @Override public long getOffHeapPuts() { + return 18; + } + + /** {@inheritDoc} */ + @Override public long getOffHeapRemovals() { + return 19; + } + + /** {@inheritDoc} */ + @Override public long getOffHeapEvictions() { + return 20; + } + + /** {@inheritDoc} */ + @Override public long getOffHeapHits() { + return 21; + } + + /** {@inheritDoc} */ + @Override public float getOffHeapHitPercentage() { + return 22; + } + + /** {@inheritDoc} */ + @Override public long getOffHeapMisses() { + return 23; + } + + /** {@inheritDoc} */ + @Override public float getOffHeapMissPercentage() { + return 24; + } + + /** {@inheritDoc} */ + @Override public long getOffHeapEntriesCount() { + return 25; + } + + /** {@inheritDoc} */ + @Override public long getOffHeapPrimaryEntriesCount() { + return 26; + } + + /** {@inheritDoc} */ + @Override public long getOffHeapBackupEntriesCount() { + return 27; + } + + /** {@inheritDoc} */ + @Override public long getOffHeapAllocatedSize() { + return 28; + } + + /** {@inheritDoc} */ + @Override public long getOffHeapMaxSize() { + return 29; + } + + /** {@inheritDoc} */ + @Override public long getSwapGets() { + return 30; + } + + /** {@inheritDoc} */ + @Override public long getSwapPuts() { + return 31; + } + + /** {@inheritDoc} */ + @Override public long getSwapRemovals() { + return 32; + } + + /** {@inheritDoc} */ + @Override public long getSwapHits() { + return 33; + } + + /** {@inheritDoc} */ + @Override public long getSwapMisses() { + return 34; + } + + /** {@inheritDoc} */ + @Override public long getSwapEntriesCount() { + return 35; + } + + /** {@inheritDoc} */ + @Override public long getSwapSize() { + return 36; + } + + /** {@inheritDoc} */ + @Override public float getSwapHitPercentage() { + return 37; + } + + /** {@inheritDoc} */ + @Override public float getSwapMissPercentage() { + return 38; + } + + /** {@inheritDoc} */ + @Override public int getSize() { + return 39; + } + + /** {@inheritDoc} */ + @Override public int getKeySize() { + return 40; + } + + /** {@inheritDoc} */ + @Override public boolean isEmpty() { + return true; + } + + /** {@inheritDoc} */ + @Override public int getDhtEvictQueueCurrentSize() { + return 41; + } + + /** {@inheritDoc} */ + @Override public int getTxThreadMapSize() { + return 42; + } + + /** {@inheritDoc} */ + @Override public int getTxXidMapSize() { + return 43; + } + + /** {@inheritDoc} */ + @Override public int getTxCommitQueueSize() { + return 44; + } + + /** {@inheritDoc} */ + @Override public int getTxPrepareQueueSize() { + return 45; + } + + /** {@inheritDoc} */ + @Override public int getTxStartVersionCountsSize() { + return 46; + } + + /** {@inheritDoc} */ + @Override public int getTxCommittedVersionsSize() { + return 47; + } + + /** {@inheritDoc} */ + @Override public int getTxRolledbackVersionsSize() { + return 48; + } + + /** {@inheritDoc} */ + @Override public int getTxDhtThreadMapSize() { + return 49; + } + + /** {@inheritDoc} */ + @Override public int getTxDhtXidMapSize() { + return 50; + } + + /** {@inheritDoc} */ + @Override public int getTxDhtCommitQueueSize() { + return 51; + } + + /** {@inheritDoc} */ + @Override public int getTxDhtPrepareQueueSize() { + return 52; + } + + /** {@inheritDoc} */ + @Override public int getTxDhtStartVersionCountsSize() { + return 53; + } + + /** {@inheritDoc} */ + @Override public int getTxDhtCommittedVersionsSize() { + return 54; + } + + /** {@inheritDoc} */ + @Override public int getTxDhtRolledbackVersionsSize() { + return 55; + } + + /** {@inheritDoc} */ + @Override public boolean isWriteBehindEnabled() { + return true; + } + + /** {@inheritDoc} */ + @Override public int getWriteBehindFlushSize() { + return 56; + } + + /** {@inheritDoc} */ + @Override public int getWriteBehindFlushThreadCount() { + return 57; + } + + /** {@inheritDoc} */ + @Override public long getWriteBehindFlushFrequency() { + return 58; + } + + /** {@inheritDoc} */ + @Override public int getWriteBehindStoreBatchSize() { + return 59; + } + + /** {@inheritDoc} */ + @Override public int getWriteBehindTotalCriticalOverflowCount() { + return 60; + } + + /** {@inheritDoc} */ + @Override public int getWriteBehindCriticalOverflowCount() { + return 61; + } + + /** {@inheritDoc} */ + @Override public int getWriteBehindErrorRetryCount() { + return 62; + } + + /** {@inheritDoc} */ + @Override public int getWriteBehindBufferSize() { + return 63; + } + + /** {@inheritDoc} */ + @Override public String getKeyType() { + return "foo"; + } + + /** {@inheritDoc} */ + @Override public String getValueType() { + return "bar"; + } + + /** {@inheritDoc} */ + @Override public boolean isStoreByValue() { + return true; + } + + /** {@inheritDoc} */ + @Override public boolean isStatisticsEnabled() { + return true; + } + + /** {@inheritDoc} */ + @Override public boolean isManagementEnabled() { + return true; + } + + /** {@inheritDoc} */ + @Override public boolean isReadThrough() { + return true; + } + + /** {@inheritDoc} */ + @Override public boolean isWriteThrough() { + return true; + } + } +} + http://git-wip-us.apache.org/repos/asf/ignite/blob/17316f93/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj index ccd7cc4..f440c25 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj @@ -68,6 +68,7 @@ + http://git-wip-us.apache.org/repos/asf/ignite/blob/17316f93/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 new file mode 100644 index 0000000..0114280 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheMetricsTest.cs @@ -0,0 +1,128 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Core.Tests.Cache +{ + using Apache.Ignite.Core.Cache; + using Apache.Ignite.Core.Impl; + using Apache.Ignite.Core.Impl.Cache; + using NUnit.Framework; + + /// + /// Tests cache metrics propagation. + /// + public class CacheMetricsTest + { + /// + /// Tests the metrics propagation. + /// + [Test] + public void TestMetricsPropagation() + { + using (var ignite = Ignition.Start(TestUtils.GetTestConfiguration())) + { + 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); + } + } + } + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/17316f93/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 296fd13..f32de21 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheSwapSpaceTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheSwapSpaceTest.cs @@ -113,7 +113,9 @@ namespace Apache.Ignite.Core.Tests.Cache var metrics = cache.GetMetrics(); Assert.AreEqual(4, metrics.OffHeapEntriesCount); // Entry takes more space than the value - Assert.AreEqual(3, metrics.OverflowSize / entrySize); // 10 - 3 - 4 = 3 + Assert.AreEqual(3, metrics.SwapEntriesCount); // 10 - 3 - 4 = 3 + Assert.AreEqual(3, metrics.OverflowSize / entrySize); + Assert.AreEqual(metrics.SwapSize, metrics.OverflowSize); } } } http://git-wip-us.apache.org/repos/asf/ignite/blob/17316f93/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs index 3c01587..9117bf8 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs @@ -26,7 +26,7 @@ namespace Apache.Ignite.Core.Cache /// The number of get requests that were satisfied by the cache. /// /// - /// The number of hits + /// The number of hits. /// long CacheHits { get; } @@ -42,7 +42,7 @@ namespace Apache.Ignite.Core.Cache /// A miss is a get request that is not satisfied. /// /// - /// The number of misses + /// The number of misses. /// long CacheMisses { get; } @@ -80,7 +80,7 @@ namespace Apache.Ignite.Core.Cache long CacheRemovals { get; } /// - /// The total number of evictions from the cache. An eviction is a removal initiated by the cache itself + /// The total number of evictions from the cache. An eviction is a removal initiated by the cache itself /// to free up space. An eviction is not treated as a removal and does not appear in the removal counts. /// /// @@ -161,6 +161,70 @@ namespace Apache.Ignite.Core.Cache long OverflowSize { get; } /// + /// The total number of get requests to the off-heap memory. + /// + /// + /// The number of gets. + /// + long OffHeapGets { get; } + + /// + /// The total number of put requests to the off-heap memory. + /// + /// + /// The number of puts. + /// + long OffHeapPuts { get; } + + /// + /// The total number of removals from the off-heap memory. This does not include evictions. + /// + /// + /// The number of removals. + /// + long OffHeapRemovals { get; } + + /// + /// The total number of evictions from the off-heap memory. + /// + /// + /// The number of evictions. + /// + long OffHeapEvictions { get; } + + /// + /// The number of get requests that were satisfied by the off-heap memory. + /// + /// + /// The off-heap hits number. + /// + long OffHeapHits { get; } + + /// + /// Gets the percentage of hits on off-heap memory. + /// + /// + /// The percentage of hits on off-heap memory. + /// + float OffHeapHitPercentage { get; } + + /// + /// A miss is a get request that is not satisfied by off-heap memory. + /// + /// + /// The off-heap misses number. + /// + long OffHeapMisses { get; } + + /// + /// Gets the percentage of misses on off-heap memory. + /// + /// + /// The percentage of misses on off-heap memory. + /// + float OffHeapMissPercentage { get; } + + /// /// Gets number of entries stored in off-heap memory. /// /// @@ -169,6 +233,22 @@ namespace Apache.Ignite.Core.Cache long OffHeapEntriesCount { get; } /// + /// Gets the number of primary entries stored in off-heap memory. + /// + /// + /// Number of primary entries stored in off-heap memory. + /// + long OffHeapPrimaryEntriesCount { get; } + + /// + /// Gets number of backup entries stored in off-heap memory. + /// + /// + /// Number of backup entries stored in off-heap memory. + /// + long OffHeapBackupEntriesCount { get; } + + /// /// Gets memory size allocated in off-heap. /// /// @@ -177,6 +257,86 @@ namespace Apache.Ignite.Core.Cache long OffHeapAllocatedSize { get; } /// + /// Gets off-heap memory maximum size. + /// + /// + /// Off-heap memory maximum size. + /// + long OffHeapMaxSize { get; } + + /// + /// The total number of get requests to the swap. + /// + /// + /// The number of gets from the swap. + /// + long SwapGets { get; } + + /// + /// The total number of put requests to the swap. + /// + /// + /// The number of puts to the swap. + /// + long SwapPuts { get; } + + /// + /// The total number of removals from the swap. + /// + /// + /// The number of removals from the swap. + /// + long SwapRemovals { get; } + + /// + /// The number of get requests that were satisfied by the swap. + /// + /// + /// The swap hits number. + /// + long SwapHits { get; } + + /// + /// A number of get requests to that were not satisfied by the swap. + /// + /// + /// The swap misses number. + /// + long SwapMisses { get; } + + /// + /// Gets number of entries stored in swap. + /// + /// + /// Number of entries stored in swap. + /// + long SwapEntriesCount { get; } + + /// + /// Gets size of swap, in bytes. + /// + /// + /// Size of swap, in bytes. + /// + long SwapSize { get; } + + /// + /// Gets the percentage of hits on swap. + /// + /// + /// The percentage of hits on swap. + /// + float SwapHitPercentage { get; } + + /// + /// Gets the percentage of misses on swap. + /// + /// + /// The percentage of misses on swap. + /// + float SwapMissPercentage { get; } + + /// /// Gets number of non-null values in the cache. /// /// @@ -329,10 +489,10 @@ namespace Apache.Ignite.Core.Cache bool IsWriteBehindEnabled { get; } /// - /// Gets the maximum size of the write-behind buffer. When the count of unique keys in write buffer exceeds - /// this value, the buffer is scheduled for write to the underlying store. - /// - /// If this value is 0, then flush is performed only on time-elapsing basis. + /// Gets the maximum size of the write-behind buffer. When the count of unique keys in write buffer exceeds + /// this value, the buffer is scheduled for write to the underlying store. + /// + /// If this value is 0, then flush is performed only on time-elapsing basis. /// /// /// Buffer size that triggers flush procedure. @@ -348,8 +508,8 @@ namespace Apache.Ignite.Core.Cache int WriteBehindFlushThreadCount { get; } /// - /// Gets the cache flush frequency. All pending operations on the underlying store will be performed - /// within time interval not less then this value. + /// Gets the cache flush frequency. All pending operations on the underlying store will be performed + /// within time interval not less then this value. /// If this value is 0, then flush is performed only when buffer size exceeds flush size. /// /// @@ -366,7 +526,7 @@ namespace Apache.Ignite.Core.Cache int WriteBehindStoreBatchSize { get; } /// - /// Gets count of write buffer overflow events since initialization. + /// Gets count of write buffer overflow events since initialization. /// Each overflow event causes the ongoing flush operation to be performed synchronously. /// /// @@ -375,7 +535,7 @@ namespace Apache.Ignite.Core.Cache int WriteBehindTotalCriticalOverflowCount { get; } /// - /// Gets count of write buffer overflow events in progress at the moment. + /// Gets count of write buffer overflow events in progress at the moment. /// Each overflow event causes the ongoing flush operation to be performed synchronously. /// /// @@ -384,8 +544,8 @@ namespace Apache.Ignite.Core.Cache int WriteBehindCriticalOverflowCount { get; } /// - /// Gets count of cache entries that are in a store-retry state. - /// An entry is assigned a store-retry state when underlying store failed due some reason + /// Gets count of cache entries that are in a store-retry state. + /// An entry is assigned a store-retry state when underlying store failed due some reason /// and cache has enough space to retain this entry till the next try. /// /// @@ -394,7 +554,7 @@ namespace Apache.Ignite.Core.Cache int WriteBehindErrorRetryCount { get; } /// - /// Gets count of entries that were processed by the write-behind store + /// Gets count of entries that were processed by the write-behind store /// and have not been flushed to the underlying store yet. /// /// @@ -420,50 +580,50 @@ namespace Apache.Ignite.Core.Cache /// /// Whether storeByValue true or storeByReference false. When true, both keys and values are stored by value. - /// - /// When false, both keys and values are stored by reference. Caches stored by reference are capable of - /// mutation by any threads holding the reference. - /// The effects are: + /// + /// When false, both keys and values are stored by reference. Caches stored by reference are capable of + /// mutation by any threads holding the reference. + /// The effects are: /// - if the key is mutated, then the key may not be retrievable or removable /// - if the value is mutated, then all threads in the JVM can potentially observe those mutations, subject /// to the normal Java Memory Model rules. - /// Storage by reference only applies to the local heap. - /// If an entry is moved off heap it will need to be transformed into a representation. - /// Any mutations that occur after transformation may not be reflected in the cache. - /// - /// When a cache is storeByValue, any mutation to the key or value does not affect the key of value - /// stored in the cache. - /// + /// Storage by reference only applies to the local heap. + /// If an entry is moved off heap it will need to be transformed into a representation. + /// Any mutations that occur after transformation may not be reflected in the cache. + /// + /// When a cache is storeByValue, any mutation to the key or value does not affect the key of value + /// stored in the cache. + /// /// The default value is true. /// /// - /// True if the cache is store by value + /// True if the cache is store by value. /// bool IsStoreByValue { get; } /// - /// Checks whether statistics collection is enabled in this cache. - /// + /// Checks whether statistics collection is enabled in this cache. + /// /// The default value is false. /// /// - /// True if statistics collection is enabled + /// True if statistics collection is enabled. /// bool IsStatisticsEnabled { get; } /// - /// Checks whether management is enabled on this cache. - /// + /// Checks whether management is enabled on this cache. + /// /// The default value is false. /// /// - /// True if management is enabled + /// True if management is enabled. /// bool IsManagementEnabled { get; } /// - /// Determines if a cache should operate in read-through mode. - /// + /// Determines if a cache should operate in read-through mode. + /// /// The default value is false /// /// @@ -472,10 +632,10 @@ namespace Apache.Ignite.Core.Cache bool IsReadThrough { get; } /// - /// Determines if a cache should operate in "write-through" mode. - /// - /// Will appropriately cause the configured CacheWriter to be invoked. - /// + /// Determines if a cache should operate in "write-through" mode. + /// + /// Will appropriately cause the configured CacheWriter to be invoked. + /// /// The default value is false /// /// http://git-wip-us.apache.org/repos/asf/ignite/blob/17316f93/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs index d42a76d6..48943b4 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs @@ -25,224 +25,523 @@ namespace Apache.Ignite.Core.Impl.Cache /// internal class CacheMetricsImpl : ICacheMetrics { + /** */ + private readonly long _cacheHits; + + /** */ + private readonly float _cacheHitPercentage; + + /** */ + private readonly long _cacheMisses; + + /** */ + private readonly float _cacheMissPercentage; + + /** */ + private readonly long _cacheGets; + + /** */ + private readonly long _cachePuts; + + /** */ + private readonly long _cacheRemovals; + + /** */ + private readonly long _cacheEvictions; + + /** */ + private readonly float _averageGetTime; + + /** */ + private readonly float _averagePutTime; + + /** */ + private readonly float _averageRemoveTime; + + /** */ + private readonly float _averageTxCommitTime; + + /** */ + private readonly float _averageTxRollbackTime; + + /** */ + private readonly long _cacheTxCommits; + + /** */ + private readonly long _cacheTxRollbacks; + + /** */ + private readonly string _cacheName; + + /** */ + private readonly long _overflowSize; + + /** */ + private readonly long _offHeapGets; + + /** */ + private readonly long _offHeapPuts; + + /** */ + private readonly long _offHeapRemovals; + + /** */ + private readonly long _offHeapEvictions; + + /** */ + private readonly long _offHeapHits; + + /** */ + private readonly float _offHeapHitPercentage; + + /** */ + private readonly long _offHeapMisses; + + /** */ + private readonly float _offHeapMissPercentage; + + /** */ + private readonly long _offHeapEntriesCount; + + /** */ + private readonly long _offHeapPrimaryEntriesCount; + + /** */ + private readonly long _offHeapBackupEntriesCount; + + /** */ + private readonly long _offHeapAllocatedSize; + + /** */ + private readonly long _offHeapMaxSize; + + /** */ + private readonly long _swapGets; + + /** */ + private readonly long _swapPuts; + + /** */ + private readonly long _swapRemovals; + + /** */ + private readonly long _swapHits; + + /** */ + private readonly long _swapMisses; + + /** */ + private readonly long _swapEntriesCount; + + /** */ + private readonly long _swapSize; + + /** */ + private readonly float _swapHitPercentage; + + /** */ + private readonly float _swapMissPercentage; + + /** */ + private readonly int _size; + + /** */ + private readonly int _keySize; + + /** */ + private readonly bool _isEmpty; + + /** */ + private readonly int _dhtEvictQueueCurrentSize; + + /** */ + private readonly int _txThreadMapSize; + + /** */ + private readonly int _txXidMapSize; + + /** */ + private readonly int _txCommitQueueSize; + + /** */ + private readonly int _txPrepareQueueSize; + + /** */ + private readonly int _txStartVersionCountsSize; + + /** */ + private readonly int _txCommittedVersionsSize; + + /** */ + private readonly int _txRolledbackVersionsSize; + + /** */ + private readonly int _txDhtThreadMapSize; + + /** */ + private readonly int _txDhtXidMapSize; + + /** */ + private readonly int _txDhtCommitQueueSize; + + /** */ + private readonly int _txDhtPrepareQueueSize; + + /** */ + private readonly int _txDhtStartVersionCountsSize; + + /** */ + private readonly int _txDhtCommittedVersionsSize; + + /** */ + private readonly int _txDhtRolledbackVersionsSize; + + /** */ + private readonly bool _isWriteBehindEnabled; + + /** */ + private readonly int _writeBehindFlushSize; + + /** */ + private readonly int _writeBehindFlushThreadCount; + + /** */ + private readonly long _writeBehindFlushFrequency; + + /** */ + private readonly int _writeBehindStoreBatchSize; + + /** */ + private readonly int _writeBehindTotalCriticalOverflowCount; + + /** */ + private readonly int _writeBehindCriticalOverflowCount; + + /** */ + private readonly int _writeBehindErrorRetryCount; + + /** */ + private readonly int _writeBehindBufferSize; + + /** */ + private readonly string _keyType; + + /** */ + private readonly string _valueType; + + /** */ + private readonly bool _isStoreByValue; + + /** */ + private readonly bool _isStatisticsEnabled; + + /** */ + private readonly bool _isManagementEnabled; + + /** */ + private readonly bool _isReadThrough; + + /** */ + private readonly bool _isWriteThrough; + /// /// Initializes a new instance of the class. /// /// The reader. public CacheMetricsImpl(IBinaryRawReader reader) { - CacheGets = reader.ReadLong(); - CachePuts = reader.ReadLong(); - CacheHits = reader.ReadLong(); - CacheMisses = reader.ReadLong(); - CacheTxCommits = reader.ReadLong(); - CacheTxRollbacks = reader.ReadLong(); - CacheEvictions = reader.ReadLong(); - CacheRemovals = reader.ReadLong(); - AveragePutTime = reader.ReadFloat(); - AverageGetTime = reader.ReadFloat(); - AverageRemoveTime = reader.ReadFloat(); - AverageTxCommitTime = reader.ReadFloat(); - AverageTxRollbackTime = reader.ReadFloat(); - CacheName = reader.ReadString(); - OverflowSize = reader.ReadLong(); - OffHeapEntriesCount = reader.ReadLong(); - OffHeapAllocatedSize = reader.ReadLong(); - Size = reader.ReadInt(); - KeySize = reader.ReadInt(); - IsEmpty = reader.ReadBoolean(); - DhtEvictQueueCurrentSize = reader.ReadInt(); - TxThreadMapSize = reader.ReadInt(); - TxXidMapSize = reader.ReadInt(); - TxCommitQueueSize = reader.ReadInt(); - TxPrepareQueueSize = reader.ReadInt(); - TxStartVersionCountsSize = reader.ReadInt(); - TxCommittedVersionsSize = reader.ReadInt(); - TxRolledbackVersionsSize = reader.ReadInt(); - TxDhtThreadMapSize = reader.ReadInt(); - TxDhtXidMapSize = reader.ReadInt(); - TxDhtCommitQueueSize = reader.ReadInt(); - TxDhtPrepareQueueSize = reader.ReadInt(); - TxDhtStartVersionCountsSize = reader.ReadInt(); - TxDhtCommittedVersionsSize = reader.ReadInt(); - TxDhtRolledbackVersionsSize = reader.ReadInt(); - IsWriteBehindEnabled = reader.ReadBoolean(); - WriteBehindFlushSize = reader.ReadInt(); - WriteBehindFlushThreadCount = reader.ReadInt(); - WriteBehindFlushFrequency = reader.ReadLong(); - WriteBehindStoreBatchSize = reader.ReadInt(); - WriteBehindTotalCriticalOverflowCount = reader.ReadInt(); - WriteBehindCriticalOverflowCount = reader.ReadInt(); - WriteBehindErrorRetryCount = reader.ReadInt(); - WriteBehindBufferSize = reader.ReadInt(); - KeyType = reader.ReadString(); - ValueType = reader.ReadString(); - IsStoreByValue = reader.ReadBoolean(); - IsStatisticsEnabled = reader.ReadBoolean(); - IsManagementEnabled = reader.ReadBoolean(); - IsReadThrough = reader.ReadBoolean(); - IsWriteThrough = reader.ReadBoolean(); - CacheHitPercentage = reader.ReadFloat(); - CacheMissPercentage = reader.ReadFloat(); + _cacheHits = reader.ReadLong(); + _cacheHitPercentage = reader.ReadFloat(); + _cacheMisses = reader.ReadLong(); + _cacheMissPercentage = reader.ReadFloat(); + _cacheGets = reader.ReadLong(); + _cachePuts = reader.ReadLong(); + _cacheRemovals = reader.ReadLong(); + _cacheEvictions = reader.ReadLong(); + _averageGetTime = reader.ReadFloat(); + _averagePutTime = reader.ReadFloat(); + _averageRemoveTime = reader.ReadFloat(); + _averageTxCommitTime = reader.ReadFloat(); + _averageTxRollbackTime = reader.ReadFloat(); + _cacheTxCommits = reader.ReadLong(); + _cacheTxRollbacks = reader.ReadLong(); + _cacheName = reader.ReadString(); + _overflowSize = reader.ReadLong(); + _offHeapGets = reader.ReadLong(); + _offHeapPuts = reader.ReadLong(); + _offHeapRemovals = reader.ReadLong(); + _offHeapEvictions = reader.ReadLong(); + _offHeapHits = reader.ReadLong(); + _offHeapHitPercentage = reader.ReadFloat(); + _offHeapMisses = reader.ReadLong(); + _offHeapMissPercentage = reader.ReadFloat(); + _offHeapEntriesCount = reader.ReadLong(); + _offHeapPrimaryEntriesCount = reader.ReadLong(); + _offHeapBackupEntriesCount = reader.ReadLong(); + _offHeapAllocatedSize = reader.ReadLong(); + _offHeapMaxSize = reader.ReadLong(); + _swapGets = reader.ReadLong(); + _swapPuts = reader.ReadLong(); + _swapRemovals = reader.ReadLong(); + _swapHits = reader.ReadLong(); + _swapMisses = reader.ReadLong(); + _swapEntriesCount = reader.ReadLong(); + _swapSize = reader.ReadLong(); + _swapHitPercentage = reader.ReadFloat(); + _swapMissPercentage = reader.ReadFloat(); + _size = reader.ReadInt(); + _keySize = reader.ReadInt(); + _isEmpty = reader.ReadBoolean(); + _dhtEvictQueueCurrentSize = reader.ReadInt(); + _txThreadMapSize = reader.ReadInt(); + _txXidMapSize = reader.ReadInt(); + _txCommitQueueSize = reader.ReadInt(); + _txPrepareQueueSize = reader.ReadInt(); + _txStartVersionCountsSize = reader.ReadInt(); + _txCommittedVersionsSize = reader.ReadInt(); + _txRolledbackVersionsSize = reader.ReadInt(); + _txDhtThreadMapSize = reader.ReadInt(); + _txDhtXidMapSize = reader.ReadInt(); + _txDhtCommitQueueSize = reader.ReadInt(); + _txDhtPrepareQueueSize = reader.ReadInt(); + _txDhtStartVersionCountsSize = reader.ReadInt(); + _txDhtCommittedVersionsSize = reader.ReadInt(); + _txDhtRolledbackVersionsSize = reader.ReadInt(); + _isWriteBehindEnabled = reader.ReadBoolean(); + _writeBehindFlushSize = reader.ReadInt(); + _writeBehindFlushThreadCount = reader.ReadInt(); + _writeBehindFlushFrequency = reader.ReadLong(); + _writeBehindStoreBatchSize = reader.ReadInt(); + _writeBehindTotalCriticalOverflowCount = reader.ReadInt(); + _writeBehindCriticalOverflowCount = reader.ReadInt(); + _writeBehindErrorRetryCount = reader.ReadInt(); + _writeBehindBufferSize = reader.ReadInt(); + _keyType = reader.ReadString(); + _valueType = reader.ReadString(); + _isStoreByValue = reader.ReadBoolean(); + _isStatisticsEnabled = reader.ReadBoolean(); + _isManagementEnabled = reader.ReadBoolean(); + _isReadThrough = reader.ReadBoolean(); + _isWriteThrough = reader.ReadBoolean(); } - /** */ - public long CacheHits { get; private set; } + /** */ + public long CacheHits { get { return _cacheHits; } } + + /** */ + public float CacheHitPercentage { get { return _cacheHitPercentage; } } + + /** */ + public long CacheMisses { get { return _cacheMisses; } } + + /** */ + public float CacheMissPercentage { get { return _cacheMissPercentage; } } + + /** */ + public long CacheGets { get { return _cacheGets; } } + + /** */ + public long CachePuts { get { return _cachePuts; } } + + /** */ + public long CacheRemovals { get { return _cacheRemovals; } } + + /** */ + public long CacheEvictions { get { return _cacheEvictions; } } + + /** */ + public float AverageGetTime { get { return _averageGetTime; } } + + /** */ + public float AveragePutTime { get { return _averagePutTime; } } + + /** */ + public float AverageRemoveTime { get { return _averageRemoveTime; } } + + /** */ + public float AverageTxCommitTime { get { return _averageTxCommitTime; } } + + /** */ + public float AverageTxRollbackTime { get { return _averageTxRollbackTime; } } + + /** */ + public long CacheTxCommits { get { return _cacheTxCommits; } } + + /** */ + public long CacheTxRollbacks { get { return _cacheTxRollbacks; } } + + /** */ + public string CacheName { get { return _cacheName; } } + + /** */ + public long OverflowSize { get { return _overflowSize; } } + + /** */ + public long OffHeapGets { get { return _offHeapGets; } } + + /** */ + public long OffHeapPuts { get { return _offHeapPuts; } } + + /** */ + public long OffHeapRemovals { get { return _offHeapRemovals; } } + + /** */ + public long OffHeapEvictions { get { return _offHeapEvictions; } } - /** */ - public float CacheHitPercentage { get; private set; } + /** */ + public long OffHeapHits { get { return _offHeapHits; } } - /** */ - public long CacheMisses { get; private set; } + /** */ + public float OffHeapHitPercentage { get { return _offHeapHitPercentage; } } - /** */ - public float CacheMissPercentage { get; private set; } + /** */ + public long OffHeapMisses { get { return _offHeapMisses; } } - /** */ - public long CacheGets { get; private set; } + /** */ + public float OffHeapMissPercentage { get { return _offHeapMissPercentage; } } - /** */ - public long CachePuts { get; private set; } + /** */ + public long OffHeapEntriesCount { get { return _offHeapEntriesCount; } } - /** */ - public long CacheRemovals { get; private set; } + /** */ + public long OffHeapPrimaryEntriesCount { get { return _offHeapPrimaryEntriesCount; } } - /** */ - public long CacheEvictions { get; private set; } + /** */ + public long OffHeapBackupEntriesCount { get { return _offHeapBackupEntriesCount; } } - /** */ - public float AverageGetTime { get; private set; } + /** */ + public long OffHeapAllocatedSize { get { return _offHeapAllocatedSize; } } - /** */ - public float AveragePutTime { get; private set; } + /** */ + public long OffHeapMaxSize { get { return _offHeapMaxSize; } } - /** */ - public float AverageRemoveTime { get; private set; } + /** */ + public long SwapGets { get { return _swapGets; } } - /** */ - public float AverageTxCommitTime { get; private set; } + /** */ + public long SwapPuts { get { return _swapPuts; } } - /** */ - public float AverageTxRollbackTime { get; private set; } + /** */ + public long SwapRemovals { get { return _swapRemovals; } } - /** */ - public long CacheTxCommits { get; private set; } + /** */ + public long SwapHits { get { return _swapHits; } } - /** */ - public long CacheTxRollbacks { get; private set; } + /** */ + public long SwapMisses { get { return _swapMisses; } } - /** */ - public string CacheName { get; private set; } + /** */ + public long SwapEntriesCount { get { return _swapEntriesCount; } } - /** */ - public long OverflowSize { get; private set; } + /** */ + public long SwapSize { get { return _swapSize; } } - /** */ - public long OffHeapEntriesCount { get; private set; } + /** */ + public float SwapHitPercentage { get { return _swapHitPercentage; } } - /** */ - public long OffHeapAllocatedSize { get; private set; } + /** */ + public float SwapMissPercentage { get { return _swapMissPercentage; } } - /** */ - public int Size { get; private set; } + /** */ + public int Size { get { return _size; } } - /** */ - public int KeySize { get; private set; } + /** */ + public int KeySize { get { return _keySize; } } - /** */ - public bool IsEmpty { get; private set; } + /** */ + public bool IsEmpty { get { return _isEmpty; } } - /** */ - public int DhtEvictQueueCurrentSize { get; private set; } + /** */ + public int DhtEvictQueueCurrentSize { get { return _dhtEvictQueueCurrentSize; } } - /** */ - public int TxThreadMapSize { get; private set; } + /** */ + public int TxThreadMapSize { get { return _txThreadMapSize; } } - /** */ - public int TxXidMapSize { get; private set; } + /** */ + public int TxXidMapSize { get { return _txXidMapSize; } } - /** */ - public int TxCommitQueueSize { get; private set; } + /** */ + public int TxCommitQueueSize { get { return _txCommitQueueSize; } } - /** */ - public int TxPrepareQueueSize { get; private set; } + /** */ + public int TxPrepareQueueSize { get { return _txPrepareQueueSize; } } - /** */ - public int TxStartVersionCountsSize { get; private set; } + /** */ + public int TxStartVersionCountsSize { get { return _txStartVersionCountsSize; } } - /** */ - public int TxCommittedVersionsSize { get; private set; } + /** */ + public int TxCommittedVersionsSize { get { return _txCommittedVersionsSize; } } - /** */ - public int TxRolledbackVersionsSize { get; private set; } + /** */ + public int TxRolledbackVersionsSize { get { return _txRolledbackVersionsSize; } } - /** */ - public int TxDhtThreadMapSize { get; private set; } + /** */ + public int TxDhtThreadMapSize { get { return _txDhtThreadMapSize; } } - /** */ - public int TxDhtXidMapSize { get; private set; } + /** */ + public int TxDhtXidMapSize { get { return _txDhtXidMapSize; } } - /** */ - public int TxDhtCommitQueueSize { get; private set; } + /** */ + public int TxDhtCommitQueueSize { get { return _txDhtCommitQueueSize; } } - /** */ - public int TxDhtPrepareQueueSize { get; private set; } + /** */ + public int TxDhtPrepareQueueSize { get { return _txDhtPrepareQueueSize; } } - /** */ - public int TxDhtStartVersionCountsSize { get; private set; } + /** */ + public int TxDhtStartVersionCountsSize { get { return _txDhtStartVersionCountsSize; } } - /** */ - public int TxDhtCommittedVersionsSize { get; private set; } + /** */ + public int TxDhtCommittedVersionsSize { get { return _txDhtCommittedVersionsSize; } } - /** */ - public int TxDhtRolledbackVersionsSize { get; private set; } + /** */ + public int TxDhtRolledbackVersionsSize { get { return _txDhtRolledbackVersionsSize; } } - /** */ - public bool IsWriteBehindEnabled { get; private set; } + /** */ + public bool IsWriteBehindEnabled { get { return _isWriteBehindEnabled; } } - /** */ - public int WriteBehindFlushSize { get; private set; } + /** */ + public int WriteBehindFlushSize { get { return _writeBehindFlushSize; } } - /** */ - public int WriteBehindFlushThreadCount { get; private set; } + /** */ + public int WriteBehindFlushThreadCount { get { return _writeBehindFlushThreadCount; } } - /** */ - public long WriteBehindFlushFrequency { get; private set; } + /** */ + public long WriteBehindFlushFrequency { get { return _writeBehindFlushFrequency; } } - /** */ - public int WriteBehindStoreBatchSize { get; private set; } + /** */ + public int WriteBehindStoreBatchSize { get { return _writeBehindStoreBatchSize; } } - /** */ - public int WriteBehindTotalCriticalOverflowCount { get; private set; } + /** */ + public int WriteBehindTotalCriticalOverflowCount { get { return _writeBehindTotalCriticalOverflowCount; } } - /** */ - public int WriteBehindCriticalOverflowCount { get; private set; } + /** */ + public int WriteBehindCriticalOverflowCount { get { return _writeBehindCriticalOverflowCount; } } - /** */ - public int WriteBehindErrorRetryCount { get; private set; } + /** */ + public int WriteBehindErrorRetryCount { get { return _writeBehindErrorRetryCount; } } - /** */ - public int WriteBehindBufferSize { get; private set; } + /** */ + public int WriteBehindBufferSize { get { return _writeBehindBufferSize; } } - /** */ - public string KeyType { get; private set; } + /** */ + public string KeyType { get { return _keyType; } } - /** */ - public string ValueType { get; private set; } + /** */ + public string ValueType { get { return _valueType; } } - /** */ - public bool IsStoreByValue { get; private set; } + /** */ + public bool IsStoreByValue { get { return _isStoreByValue; } } - /** */ - public bool IsStatisticsEnabled { get; private set; } + /** */ + public bool IsStatisticsEnabled { get { return _isStatisticsEnabled; } } - /** */ - public bool IsManagementEnabled { get; private set; } + /** */ + public bool IsManagementEnabled { get { return _isManagementEnabled; } } - /** */ - public bool IsReadThrough { get; private set; } + /** */ + public bool IsReadThrough { get { return _isReadThrough; } } - /** */ - public bool IsWriteThrough { get; private set; } + /** */ + public bool IsWriteThrough { get { return _isWriteThrough; } } } } \ No newline at end of file