http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c7a9a1db/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsAdapter.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsAdapter.java
index 0000000,0000000..0752544
new file mode 100644
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsAdapter.java
@@@ -1,0 -1,0 +1,551 @@@
++/*
++ * Licensed to the Apache Software Foundation (ASF) under one or more
++ * contributor license agreements.  See the NOTICE file distributed with
++ * this work for additional information regarding copyright ownership.
++ * The ASF licenses this file to You under the Apache License, Version 2.0
++ * (the "License"); you may not use this file except in compliance with
++ * the License.  You may obtain a copy of the License at
++ *
++ *      http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++
++package org.apache.ignite.internal.processors.cache;
++
++import org.apache.ignite.*;
++import org.apache.ignite.cache.*;
++import org.apache.ignite.internal.util.tostring.*;
++import org.apache.ignite.internal.util.typedef.internal.*;
++
++import java.util.concurrent.atomic.*;
++
++/**
++ * Adapter for cache metrics.
++ */
++public class CacheMetricsAdapter implements CacheMetrics {
++    /** */
++    private static final long NANOS_IN_MICROSECOND = 1000L;
++
++    /** Number of reads. */
++    private AtomicLong reads = new AtomicLong();
++
++    /** Number of writes. */
++    private AtomicLong writes = new AtomicLong();
++
++    /** Number of hits. */
++    private AtomicLong hits = new AtomicLong();
++
++    /** Number of misses. */
++    private AtomicLong misses = new AtomicLong();
++
++    /** Number of transaction commits. */
++    private AtomicLong txCommits = new AtomicLong();
++
++    /** Number of transaction rollbacks. */
++    private AtomicLong txRollbacks = new AtomicLong();
++
++    /** Number of evictions. */
++    private AtomicLong evictCnt = new AtomicLong();
++
++    /** Number of removed entries. */
++    private AtomicLong rmCnt = new AtomicLong();
++
++    /** Put time taken nanos. */
++    private AtomicLong putTimeNanos = new AtomicLong();
++
++    /** Get time taken nanos. */
++    private AtomicLong getTimeNanos = new AtomicLong();
++
++    /** Remove time taken nanos. */
++    private AtomicLong removeTimeNanos = new AtomicLong();
++
++    /** Commit transaction time taken nanos. */
++    private AtomicLong commitTimeNanos = new AtomicLong();
++
++    /** Commit transaction time taken nanos. */
++    private AtomicLong rollbackTimeNanos = new AtomicLong();
++
++    /** Cache metrics. */
++    @GridToStringExclude
++    private transient CacheMetricsAdapter delegate;
++
++    /** Cache context. */
++    private GridCacheContext<?, ?> cctx;
++
++    /** DHT context. */
++    private GridCacheContext<?, ?> dhtCtx;
++
++    /** Write-behind store, if configured. */
++    private GridCacheWriteBehindStore store;
++
++    /**
++     * Creates cache metrics;
++     *
++     * @param cctx Cache context.
++     */
++    public CacheMetricsAdapter(GridCacheContext<?, ?> cctx) {
++        assert cctx != null;
++
++        this.cctx = cctx;
++
++        if (cctx.isNear())
++            dhtCtx = cctx.near().dht().context();
++
++        if (cctx.store().store() instanceof GridCacheWriteBehindStore)
++            store = (GridCacheWriteBehindStore)cctx.store().store();
++
++        delegate = null;
++    }
++
++    /**
++     * @param delegate Metrics to delegate to.
++     */
++    public void delegate(CacheMetricsAdapter delegate) {
++        this.delegate = delegate;
++    }
++
++
++    /** {@inheritDoc} */
++    @Override public String name() {
++        return cctx.name();
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getOverflowSize() {
++        try {
++            return cctx.cache().overflowSize();
++        }
++        catch (IgniteCheckedException ignored) {
++            return -1;
++        }
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getOffHeapEntriesCount() {
++        return cctx.cache().offHeapEntriesCount();
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getOffHeapAllocatedSize() {
++        return cctx.cache().offHeapAllocatedSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getSize() {
++        return cctx.cache().size();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getKeySize() {
++        return cctx.cache().size();
++    }
++
++    /** {@inheritDoc} */
++    @Override public boolean isEmpty() {
++        return cctx.cache().isEmpty();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getDhtEvictQueueCurrentSize() {
++        return cctx.isNear() ?
++                dhtCtx != null ? dhtCtx.evicts().evictQueueSize() : -1
++                : cctx.evicts().evictQueueSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxCommitQueueSize() {
++        return cctx.tm().commitQueueSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxThreadMapSize() {
++        return cctx.tm().threadMapSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxXidMapSize() {
++        return cctx.tm().idMapSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxPrepareQueueSize() {
++        return cctx.tm().prepareQueueSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxStartVersionCountsSize() {
++        return cctx.tm().startVersionCountsSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxCommittedVersionsSize() {
++        return cctx.tm().committedVersionsSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxRolledbackVersionsSize() {
++        return cctx.tm().rolledbackVersionsSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxDhtThreadMapSize() {
++        return cctx.isNear() && dhtCtx != null ? dhtCtx.tm().threadMapSize() 
: -1;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxDhtXidMapSize() {
++        return cctx.isNear() && dhtCtx != null ? dhtCtx.tm().idMapSize() : -1;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxDhtCommitQueueSize() {
++        return cctx.isNear() && dhtCtx != null ? 
dhtCtx.tm().commitQueueSize() : -1;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxDhtPrepareQueueSize() {
++        return cctx.isNear() && dhtCtx != null ? 
dhtCtx.tm().prepareQueueSize() : -1;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxDhtStartVersionCountsSize() {
++        return cctx.isNear() && dhtCtx != null ? 
dhtCtx.tm().startVersionCountsSize() : -1;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxDhtCommittedVersionsSize() {
++        return cctx.isNear() && dhtCtx != null ? 
dhtCtx.tm().committedVersionsSize() : -1;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxDhtRolledbackVersionsSize() {
++        return cctx.isNear() && dhtCtx != null ? 
dhtCtx.tm().rolledbackVersionsSize() : -1;
++    }
++
++    /** {@inheritDoc} */
++    @Override public boolean isWriteBehindEnabled() {
++        return store != null;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getWriteBehindFlushSize() {
++        return store != null ? store.getWriteBehindFlushSize() : -1;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getWriteBehindFlushThreadCount() {
++        return store != null ? store.getWriteBehindFlushThreadCount() : -1;
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getWriteBehindFlushFrequency() {
++        return store != null ? store.getWriteBehindFlushFrequency() : -1;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getWriteBehindStoreBatchSize() {
++        return store != null ? store.getWriteBehindStoreBatchSize() : -1;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getWriteBehindTotalCriticalOverflowCount() {
++        return store != null ? 
store.getWriteBehindTotalCriticalOverflowCount() : -1;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getWriteBehindCriticalOverflowCount() {
++        return store != null ? store.getWriteBehindCriticalOverflowCount() : 
-1;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getWriteBehindErrorRetryCount() {
++        return store != null ? store.getWriteBehindErrorRetryCount() : -1;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getWriteBehindBufferSize() {
++        return store != null ? store.getWriteBehindBufferSize() : -1;
++    }
++
++    /** {@inheritDoc} */
++    @Override public float getAverageTxCommitTime() {
++        long timeNanos = commitTimeNanos.get();
++        long commitsCnt = txCommits.get();
++
++        if (timeNanos == 0 || commitsCnt == 0)
++            return 0;
++
++        return ((1f * timeNanos) / commitsCnt) / NANOS_IN_MICROSECOND;
++    }
++
++    /** {@inheritDoc} */
++    @Override public float getAverageTxRollbackTime() {
++        long timeNanos = rollbackTimeNanos.get();
++        long rollbacksCnt = txRollbacks.get();
++
++        if (timeNanos == 0 || rollbacksCnt == 0)
++            return 0;
++
++        return ((1f * timeNanos) / rollbacksCnt) / NANOS_IN_MICROSECOND;
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getCacheTxCommits() {
++        return txCommits.get();
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getCacheTxRollbacks() {
++        return txRollbacks.get();
++    }
++
++    /**
++     * Clear metrics.
++     */
++    public void clear() {
++        reads.set(0);
++        writes.set(0);
++        rmCnt.set(0);
++        hits.set(0);
++        misses.set(0);
++        evictCnt.set(0);
++        txCommits.set(0);
++        txRollbacks.set(0);
++        putTimeNanos.set(0);
++        removeTimeNanos.set(0);
++        getTimeNanos.set(0);
++        commitTimeNanos.set(0);
++        rollbackTimeNanos.set(0);
++
++        if (delegate != null)
++            delegate.clear();
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getCacheHits() {
++        return hits.get();
++    }
++
++    /** {@inheritDoc} */
++    @Override public float getCacheHitPercentage() {
++        long hits0 = hits.get();
++        long gets0 = reads.get();
++
++        if (hits0 == 0)
++            return 0;
++
++        return (float) hits0 / gets0 * 100.0f;
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getCacheMisses() {
++        return misses.get();
++    }
++
++    /** {@inheritDoc} */
++    @Override public float getCacheMissPercentage() {
++        long misses0 = misses.get();
++        long reads0 = reads.get();
++
++        if (misses0 == 0) {
++            return 0;
++        }
++
++        return (float) misses0 / reads0 * 100.0f;
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getCacheGets() {
++        return reads.get();
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getCachePuts() {
++        return writes.get();
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getCacheRemovals() {
++        return rmCnt.get();
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getCacheEvictions() {
++        return evictCnt.get();
++    }
++
++    /** {@inheritDoc} */
++    @Override public float getAverageGetTime() {
++        long timeNanos = getTimeNanos.get();
++        long readsCnt = reads.get();
++
++        if (timeNanos == 0 || readsCnt == 0)
++            return 0;
++
++        return ((1f * timeNanos) / readsCnt) / NANOS_IN_MICROSECOND;
++    }
++
++    /** {@inheritDoc} */
++    @Override public float getAveragePutTime() {
++        long timeNanos = putTimeNanos.get();
++        long putsCnt = writes.get();
++
++        if (timeNanos == 0 || putsCnt == 0)
++            return 0;
++
++        return ((1f * timeNanos) / putsCnt) / NANOS_IN_MICROSECOND;
++    }
++
++    /** {@inheritDoc} */
++    @Override public float getAverageRemoveTime() {
++        long timeNanos = removeTimeNanos.get();
++        long removesCnt = rmCnt.get();
++
++        if (timeNanos == 0 || removesCnt == 0)
++            return 0;
++
++        return ((1f * timeNanos) / removesCnt) / NANOS_IN_MICROSECOND;
++    }
++
++    /**
++     * Cache read callback.
++     * @param isHit Hit or miss flag.
++     */
++    public void onRead(boolean isHit) {
++        reads.incrementAndGet();
++
++        if (isHit)
++            hits.incrementAndGet();
++        else
++            misses.incrementAndGet();
++
++        if (delegate != null)
++            delegate.onRead(isHit);
++    }
++
++    /**
++     * Cache write callback.
++     */
++    public void onWrite() {
++        writes.incrementAndGet();
++
++        if (delegate != null)
++            delegate.onWrite();
++    }
++
++    /**
++     * Cache remove callback.
++     */
++    public void onRemove(){
++        rmCnt.incrementAndGet();
++
++        if (delegate != null)
++            delegate.onRemove();
++    }
++
++    /**
++     * Cache remove callback.
++     */
++    public void onEvict() {
++        evictCnt.incrementAndGet();
++
++        if (delegate != null)
++            delegate.onEvict();
++    }
++
++    /**
++     * Transaction commit callback.
++     */
++    public void onTxCommit(long duration) {
++        txCommits.incrementAndGet();
++        commitTimeNanos.addAndGet(duration);
++
++        if (delegate != null) {
++            delegate.onTxCommit(duration);
++        }
++    }
++
++    /**
++     * Transaction rollback callback.
++     */
++    public void onTxRollback(long duration) {
++        txRollbacks.incrementAndGet();
++        rollbackTimeNanos.addAndGet(duration);
++
++        if (delegate != null)
++            delegate.onTxRollback(duration);
++    }
++
++
++    /**
++     * Increments the get time accumulator.
++     *
++     * @param duration the time taken in nanoseconds.
++     */
++    public void addGetTimeNanos(long duration) {
++        getTimeNanos.addAndGet(duration);
++
++        if (delegate != null)
++            delegate.addGetTimeNanos(duration);
++    }
++
++    /**
++     * Increments the put time accumulator.
++     *
++     * @param duration the time taken in nanoseconds.
++     */
++    public void addPutTimeNanos(long duration) {
++        putTimeNanos.addAndGet(duration);
++
++        if (delegate != null)
++            delegate.addPutTimeNanos(duration);
++    }
++
++    /**
++     * Increments the remove time accumulator.
++     *
++     * @param duration the time taken in nanoseconds.
++     */
++    public void addRemoveTimeNanos(long duration) {
++        removeTimeNanos.addAndGet(duration);
++
++        if (delegate != null)
++            delegate.addRemoveTimeNanos(duration);
++    }
++
++    /**
++     * Increments remove and get time accumulators.
++     *
++     * @param duration the time taken in nanoseconds.
++     */
++    public void addRemoveAndGetTimeNanos(long duration) {
++        removeTimeNanos.addAndGet(duration);
++        getTimeNanos.addAndGet(duration);
++
++        if (delegate != null)
++            delegate.addRemoveAndGetTimeNanos(duration);
++    }
++
++    /**
++     * Increments put and get time accumulators.
++     *
++     * @param duration the time taken in nanoseconds.
++     */
++    public void addPutAndGetTimeNanos(long duration) {
++        putTimeNanos.addAndGet(duration);
++        getTimeNanos.addAndGet(duration);
++
++        if (delegate != null)
++            delegate.addPutAndGetTimeNanos(duration);
++    }
++
++    /** {@inheritDoc} */
++    @Override public String toString() {
++        return S.toString(CacheMetricsAdapter.class, this);
++    }
++}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c7a9a1db/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java
index 0000000,0000000..23c757c
new file mode 100644
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java
@@@ -1,0 -1,0 +1,559 @@@
++/*
++ * Licensed to the Apache Software Foundation (ASF) under one or more
++ * contributor license agreements.  See the NOTICE file distributed with
++ * this work for additional information regarding copyright ownership.
++ * The ASF licenses this file to You under the Apache License, Version 2.0
++ * (the "License"); you may not use this file except in compliance with
++ * the License.  You may obtain a copy of the License at
++ *
++ *      http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++
++package org.apache.ignite.internal.processors.cache;
++
++import org.apache.ignite.cache.*;
++import org.apache.ignite.internal.util.typedef.internal.*;
++
++import java.io.*;
++
++/**
++ * Metrics snapshot.
++ */
++class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
++    /** */
++    private static final long serialVersionUID = 0L;
++
++    /** Number of reads. */
++    private long reads = 0;
++
++    /** Number of puts. */
++    private long puts = 0;
++
++    /** Number of hits. */
++    private long hits = 0;
++
++    /** Number of misses. */
++    private long misses = 0;
++
++    /** Number of transaction commits. */
++    private long txCommits = 0;
++
++    /** Number of transaction rollbacks. */
++    private long txRollbacks = 0;
++
++    /** Number of evictions. */
++    private long evicts = 0;
++
++    /** Number of removed entries. */
++    private long removes = 0;
++
++    /** Put time taken nanos. */
++    private float putAvgTimeNanos = 0;
++
++    /** Get time taken nanos. */
++    private float getAvgTimeNanos = 0;
++
++    /** Remove time taken nanos. */
++    private float removeAvgTimeNanos = 0;
++
++    /** Commit transaction time taken nanos. */
++    private float commitAvgTimeNanos = 0;
++
++    /** Commit transaction time taken nanos. */
++    private float rollbackAvgTimeNanos = 0;
++
++    /** Cache name */
++    private String cacheName;
++
++    /** Number of entries that was swapped to disk. */
++    private long overflowSize;
++
++    /** Number of entries stored in off-heap memory. */
++    private long offHeapEntriesCount;
++
++    /** Memory size allocated in off-heap. */
++    private long offHeapAllocatedSize;
++
++    /** Number of non-{@code null} values in the cache. */
++    private int size;
++
++    /** Number of keys in the cache, possibly with {@code null} values. */
++    private int keySize;
++
++    /** Cache is empty. */
++    private boolean isEmpty;
++
++    /** Gets current size of evict queue used to batch up evictions. */
++    private int dhtEvictQueueCurrentSize;
++
++    /** Transaction per-thread map size. */
++    private int txThreadMapSize;
++
++    /** Transaction per-Xid map size. */
++    private int txXidMapSize;
++
++    /** Committed transaction queue size. */
++    private int txCommitQueueSize;
++
++    /** Prepared transaction queue size. */
++    private int txPrepareQueueSize;
++
++    /** Start version counts map size. */
++    private int txStartVersionCountsSize;
++
++    /** Number of cached committed transaction IDs. */
++    private int txCommittedVersionsSize;
++
++    /** Number of cached rolled back transaction IDs. */
++    private int txRolledbackVersionsSize;
++
++    /** DHT thread map size. */
++    private int txDhtThreadMapSize;
++
++    /** Transaction DHT per-Xid map size. */
++    private int txDhtXidMapSize;
++
++    /** Committed DHT transaction queue size. */
++    private int txDhtCommitQueueSize;
++
++    /** Prepared DHT transaction queue size. */
++    private int txDhtPrepareQueueSize;
++
++    /** DHT start version counts map size. */
++    private int txDhtStartVersionCountsSize;
++
++    /** Number of cached committed DHT transaction IDs. */
++    private int txDhtCommittedVersionsSize;
++
++    /** Number of cached rolled back DHT transaction IDs. */
++    private int txDhtRolledbackVersionsSize;
++
++    /** Write-behind is enabled. */
++    private boolean isWriteBehindEnabled;
++
++    /** Buffer size that triggers flush procedure. */
++    private int writeBehindFlushSize;
++
++    /** Count of worker threads. */
++    private int writeBehindFlushThreadCount;
++
++    /** Flush frequency in milliseconds. */
++    private long writeBehindFlushFrequency;
++
++    /** Maximum size of batch. */
++    private int writeBehindStoreBatchSize;
++
++    /** Count of cache overflow events since start. */
++    private int writeBehindTotalCriticalOverflowCount;
++
++    /** Count of cache overflow events since start. */
++    private int writeBehindCriticalOverflowCount;
++
++    /** Count of entries in store-retry state. */
++    private int writeBehindErrorRetryCount;
++
++    /** Total count of entries in cache store internal buffer. */
++    private int writeBehindBufferSize;
++
++    /**
++     * Create snapshot for given metrics.
++     *
++     * @param m Cache metrics.
++     */
++    public CacheMetricsSnapshot(CacheMetrics m) {
++        reads = m.getCacheGets();
++        puts = m.getCachePuts();
++        hits = m.getCacheHits();
++        misses = m.getCacheMisses();
++        txCommits = m.getCacheTxCommits();
++        txRollbacks = m.getCacheTxRollbacks();
++        evicts = m.getCacheEvictions();
++        removes = m.getCacheRemovals();
++
++        putAvgTimeNanos = m.getAveragePutTime();
++        getAvgTimeNanos = m.getAverageGetTime();
++        removeAvgTimeNanos = m.getAverageRemoveTime();
++        commitAvgTimeNanos = m.getAverageTxCommitTime();
++        rollbackAvgTimeNanos = m.getAverageTxRollbackTime();
++
++        cacheName = m.name();
++        overflowSize = m.getOverflowSize();
++        offHeapEntriesCount = m.getOffHeapEntriesCount();
++        offHeapAllocatedSize = m.getOffHeapAllocatedSize();
++        size = m.getSize();
++        keySize = m.getKeySize();
++        isEmpty = m.isEmpty();
++        dhtEvictQueueCurrentSize = m.getDhtEvictQueueCurrentSize();
++        txThreadMapSize = m.getTxThreadMapSize();
++        txXidMapSize = m.getTxXidMapSize();
++        txCommitQueueSize = m.getTxCommitQueueSize();
++        txPrepareQueueSize = m.getTxPrepareQueueSize();
++        txStartVersionCountsSize = m.getTxStartVersionCountsSize();
++        txCommittedVersionsSize = m.getTxCommittedVersionsSize();
++        txRolledbackVersionsSize = m.getTxRolledbackVersionsSize();
++        txDhtThreadMapSize = m.getTxDhtThreadMapSize();
++        txDhtXidMapSize = m.getTxDhtXidMapSize();
++        txDhtCommitQueueSize = m.getTxDhtCommitQueueSize();
++        txDhtPrepareQueueSize = m.getTxDhtPrepareQueueSize();
++        txDhtStartVersionCountsSize = m.getTxDhtStartVersionCountsSize();
++        txDhtCommittedVersionsSize = m.getTxDhtCommittedVersionsSize();
++        txDhtRolledbackVersionsSize = m.getTxDhtRolledbackVersionsSize();
++        isWriteBehindEnabled = m.isWriteBehindEnabled();
++        writeBehindFlushSize = m.getWriteBehindFlushSize();
++        writeBehindFlushThreadCount = m.getWriteBehindFlushThreadCount();
++        writeBehindFlushFrequency = m.getWriteBehindFlushFrequency();
++        writeBehindStoreBatchSize = m.getWriteBehindStoreBatchSize();
++        writeBehindTotalCriticalOverflowCount = 
m.getWriteBehindTotalCriticalOverflowCount();
++        writeBehindCriticalOverflowCount = 
m.getWriteBehindCriticalOverflowCount();
++        writeBehindErrorRetryCount = m.getWriteBehindErrorRetryCount();
++        writeBehindBufferSize = m.getWriteBehindBufferSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getCacheHits() {
++        return hits;
++    }
++
++    /** {@inheritDoc} */
++    @Override public float getCacheHitPercentage() {
++        if (hits == 0 || reads == 0)
++            return 0;
++
++        return (float) hits / reads * 100.0f;
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getCacheMisses() {
++        return misses;
++    }
++
++    /** {@inheritDoc} */
++    @Override public float getCacheMissPercentage() {
++        if (misses == 0 || reads == 0) {
++            return 0;
++        }
++
++        return (float) misses / reads * 100.0f;
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getCacheGets() {
++        return reads;
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getCachePuts() {
++        return puts;
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getCacheRemovals() {
++        return removes;
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getCacheEvictions() {
++        return evicts;
++    }
++
++    /** {@inheritDoc} */
++    @Override public float getAverageGetTime() {
++        return getAvgTimeNanos;
++    }
++
++    /** {@inheritDoc} */
++    @Override public float getAveragePutTime() {
++        return putAvgTimeNanos;
++    }
++
++    /** {@inheritDoc} */
++    @Override public float getAverageRemoveTime() {
++        return removeAvgTimeNanos;
++    }
++
++    /** {@inheritDoc} */
++    @Override public float getAverageTxCommitTime() {
++        return commitAvgTimeNanos;
++    }
++
++    /** {@inheritDoc} */
++    @Override public float getAverageTxRollbackTime() {
++        return rollbackAvgTimeNanos;
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getCacheTxCommits() {
++        return txCommits;
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getCacheTxRollbacks() {
++        return txRollbacks;
++    }
++
++    /** {@inheritDoc} */
++    @Override public String name() {
++        return cacheName;
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getOverflowSize() {
++        return overflowSize;
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getOffHeapEntriesCount() {
++        return offHeapEntriesCount;
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getOffHeapAllocatedSize() {
++        return offHeapAllocatedSize;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getSize() {
++        return size;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getKeySize() {
++        return keySize;
++    }
++
++    /** {@inheritDoc} */
++    @Override public boolean isEmpty() {
++        return isEmpty;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getDhtEvictQueueCurrentSize() {
++        return dhtEvictQueueCurrentSize;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxThreadMapSize() {
++        return txThreadMapSize;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxXidMapSize() {
++        return txXidMapSize;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxCommitQueueSize() {
++        return txCommitQueueSize;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxPrepareQueueSize() {
++        return txPrepareQueueSize;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxStartVersionCountsSize() {
++        return txStartVersionCountsSize;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxCommittedVersionsSize() {
++        return txCommittedVersionsSize;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxRolledbackVersionsSize() {
++        return txRolledbackVersionsSize;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxDhtThreadMapSize() {
++        return txDhtThreadMapSize;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxDhtXidMapSize() {
++        return txDhtXidMapSize;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxDhtCommitQueueSize() {
++        return txDhtCommitQueueSize;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxDhtPrepareQueueSize() {
++        return txDhtPrepareQueueSize;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxDhtStartVersionCountsSize() {
++        return txDhtStartVersionCountsSize;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxDhtCommittedVersionsSize() {
++        return txDhtCommittedVersionsSize;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxDhtRolledbackVersionsSize() {
++        return txDhtRolledbackVersionsSize;
++    }
++
++    /** {@inheritDoc} */
++    @Override public boolean isWriteBehindEnabled() {
++        return isWriteBehindEnabled;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getWriteBehindFlushSize() {
++        return writeBehindFlushSize;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getWriteBehindFlushThreadCount() {
++        return writeBehindFlushThreadCount;
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getWriteBehindFlushFrequency() {
++        return writeBehindFlushFrequency;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getWriteBehindStoreBatchSize() {
++        return writeBehindStoreBatchSize;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getWriteBehindTotalCriticalOverflowCount() {
++        return writeBehindTotalCriticalOverflowCount;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getWriteBehindCriticalOverflowCount() {
++        return writeBehindCriticalOverflowCount;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getWriteBehindErrorRetryCount() {
++        return writeBehindErrorRetryCount;
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getWriteBehindBufferSize() {
++        return writeBehindBufferSize;
++    }
++
++    /** {@inheritDoc} */
++    @Override public void writeExternal(ObjectOutput out) throws IOException {
++        out.writeLong(reads);
++        out.writeLong(puts);
++        out.writeLong(hits);
++        out.writeLong(misses);
++        out.writeLong(txCommits);
++        out.writeLong(txRollbacks);
++        out.writeLong(removes);
++        out.writeLong(evicts);
++
++        out.writeFloat(putAvgTimeNanos);
++        out.writeFloat(getAvgTimeNanos);
++        out.writeFloat(removeAvgTimeNanos);
++        out.writeFloat(commitAvgTimeNanos);
++        out.writeFloat(rollbackAvgTimeNanos);
++
++        out.writeObject(cacheName);
++        out.writeLong(overflowSize);
++        out.writeLong(offHeapEntriesCount);
++        out.writeLong(offHeapAllocatedSize);
++        out.writeInt(size);
++        out.writeInt(keySize);
++        out.writeBoolean(isEmpty);
++        out.writeInt(dhtEvictQueueCurrentSize);
++        out.writeInt(txThreadMapSize);
++        out.writeInt(txXidMapSize);
++        out.writeInt(txCommitQueueSize);
++        out.writeInt(txPrepareQueueSize);
++        out.writeInt(txStartVersionCountsSize);
++        out.writeInt(txCommittedVersionsSize);
++        out.writeInt(txRolledbackVersionsSize);
++        out.writeInt(txDhtThreadMapSize);
++        out.writeInt(txDhtXidMapSize);
++        out.writeInt(txDhtCommitQueueSize);
++        out.writeInt(txDhtPrepareQueueSize);
++        out.writeInt(txDhtStartVersionCountsSize);
++        out.writeInt(txDhtCommittedVersionsSize);
++        out.writeInt(txDhtRolledbackVersionsSize);
++        out.writeBoolean(isWriteBehindEnabled);
++        out.writeInt(writeBehindFlushSize);
++        out.writeInt(writeBehindFlushThreadCount);
++        out.writeLong(writeBehindFlushFrequency);
++        out.writeInt(writeBehindStoreBatchSize);
++        out.writeInt(writeBehindTotalCriticalOverflowCount);
++        out.writeInt(writeBehindCriticalOverflowCount);
++        out.writeInt(writeBehindErrorRetryCount);
++        out.writeInt(writeBehindBufferSize);
++    }
++
++    /** {@inheritDoc} */
++    @Override public void readExternal(ObjectInput in) throws IOException, 
ClassNotFoundException {
++        reads = in.readLong();
++        puts = in.readLong();
++        hits = in.readLong();
++        misses = in.readLong();
++        txCommits = in.readLong();
++        txRollbacks = in.readLong();
++        removes = in.readLong();
++        evicts = in.readLong();
++
++        putAvgTimeNanos = in.readFloat();
++        getAvgTimeNanos = in.readFloat();
++        removeAvgTimeNanos = in.readFloat();
++        commitAvgTimeNanos = in.readFloat();
++        rollbackAvgTimeNanos = in.readFloat();
++
++        cacheName = (String)in.readObject();
++        overflowSize = in.readLong();
++        offHeapEntriesCount = in.readLong();
++        offHeapAllocatedSize = in.readLong();
++        size = in.readInt();
++        keySize = in.readInt();
++        isEmpty = in.readBoolean();
++        dhtEvictQueueCurrentSize = in.readInt();
++        txThreadMapSize = in.readInt();
++        txXidMapSize = in.readInt();
++        txCommitQueueSize = in.readInt();
++        txPrepareQueueSize = in.readInt();
++        txStartVersionCountsSize = in.readInt();
++        txCommittedVersionsSize = in.readInt();
++        txRolledbackVersionsSize = in.readInt();
++        txDhtThreadMapSize = in.readInt();
++        txDhtXidMapSize = in.readInt();
++        txDhtCommitQueueSize = in.readInt();
++        txDhtPrepareQueueSize = in.readInt();
++        txDhtStartVersionCountsSize = in.readInt();
++        txDhtCommittedVersionsSize = in.readInt();
++        txDhtRolledbackVersionsSize = in.readInt();
++        isWriteBehindEnabled = in.readBoolean();
++        writeBehindFlushSize = in.readInt();
++        writeBehindFlushThreadCount = in.readInt();
++        writeBehindFlushFrequency = in.readLong();
++        writeBehindStoreBatchSize = in.readInt();
++        writeBehindTotalCriticalOverflowCount = in.readInt();
++        writeBehindCriticalOverflowCount = in.readInt();
++        writeBehindErrorRetryCount = in.readInt();
++        writeBehindBufferSize = in.readInt();
++    }
++
++    /** {@inheritDoc} */
++    @Override public String toString() {
++        return S.toString(CacheMetricsSnapshot.class, this);
++    }
++}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c7a9a1db/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMxBeanImpl.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMxBeanImpl.java
index 0000000,0000000..2c23594
new file mode 100644
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMxBeanImpl.java
@@@ -1,0 -1,0 +1,275 @@@
++/*
++ * Licensed to the Apache Software Foundation (ASF) under one or more
++ * contributor license agreements.  See the NOTICE file distributed with
++ * this work for additional information regarding copyright ownership.
++ * The ASF licenses this file to You under the Apache License, Version 2.0
++ * (the "License"); you may not use this file except in compliance with
++ * the License.  You may obtain a copy of the License at
++ *
++ *      http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++
++package org.apache.ignite.internal.processors.cache;
++
++import org.apache.ignite.*;
++import org.apache.ignite.cache.*;
++
++/**
++ * Management bean that provides access to {@link IgniteCache IgniteCache}.
++ */
++class CacheMxBeanImpl implements IgniteCacheMxBean {
++    /** Cache. */
++    private GridCacheAdapter<?, ?> cache;
++
++    /**
++     * Creates MBean;
++     *
++     * @param cache Cache.
++     */
++    CacheMxBeanImpl(GridCacheAdapter<?, ?> cache) {
++        assert cache != null;
++
++        this.cache = cache;
++    }
++
++    /** {@inheritDoc} */
++    @Override public String name() {
++        return cache.metrics0().name();
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getOverflowSize() {
++        return cache.metrics0().getOverflowSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getOffHeapEntriesCount() {
++        return cache.metrics0().getOffHeapEntriesCount();
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getOffHeapAllocatedSize() {
++        return cache.metrics0().getOffHeapAllocatedSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getSize() {
++        return cache.metrics0().getSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getKeySize() {
++        return cache.metrics0().getKeySize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public boolean isEmpty() {
++        return cache.metrics0().isEmpty();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getDhtEvictQueueCurrentSize() {
++        return cache.metrics0().getDhtEvictQueueCurrentSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxCommitQueueSize() {
++        return cache.metrics0().getTxCommitQueueSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxThreadMapSize() {
++        return cache.metrics0().getTxThreadMapSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxXidMapSize() {
++        return cache.metrics0().getTxXidMapSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxPrepareQueueSize() {
++        return cache.metrics0().getTxPrepareQueueSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxStartVersionCountsSize() {
++        return cache.metrics0().getTxStartVersionCountsSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxCommittedVersionsSize() {
++        return cache.metrics0().getTxCommittedVersionsSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxRolledbackVersionsSize() {
++        return cache.metrics0().getTxRolledbackVersionsSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxDhtThreadMapSize() {
++        return cache.metrics0().getTxDhtThreadMapSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxDhtXidMapSize() {
++        return cache.metrics0().getTxDhtXidMapSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxDhtCommitQueueSize() {
++        return cache.metrics0().getTxDhtCommitQueueSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxDhtPrepareQueueSize() {
++        return cache.metrics0().getTxDhtPrepareQueueSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxDhtStartVersionCountsSize() {
++        return cache.metrics0().getTxDhtStartVersionCountsSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxDhtCommittedVersionsSize() {
++        return cache.metrics0().getTxDhtCommittedVersionsSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getTxDhtRolledbackVersionsSize() {
++        return cache.metrics0().getTxDhtRolledbackVersionsSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public boolean isWriteBehindEnabled() {
++        return cache.metrics0().isWriteBehindEnabled();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getWriteBehindFlushSize() {
++        return cache.metrics0().getWriteBehindFlushSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getWriteBehindFlushThreadCount() {
++        return cache.metrics0().getWriteBehindFlushThreadCount();
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getWriteBehindFlushFrequency() {
++        return cache.metrics0().getWriteBehindFlushFrequency();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getWriteBehindStoreBatchSize() {
++        return cache.metrics0().getWriteBehindStoreBatchSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getWriteBehindTotalCriticalOverflowCount() {
++        return cache.metrics0().getWriteBehindTotalCriticalOverflowCount();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getWriteBehindCriticalOverflowCount() {
++        return cache.metrics0().getWriteBehindCriticalOverflowCount();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getWriteBehindErrorRetryCount() {
++        return cache.metrics0().getWriteBehindErrorRetryCount();
++    }
++
++    /** {@inheritDoc} */
++    @Override public int getWriteBehindBufferSize() {
++        return cache.metrics0().getWriteBehindBufferSize();
++    }
++
++    /** {@inheritDoc} */
++    @Override public void clear() {
++        cache.metrics0().clear();
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getCacheHits() {
++        return cache.metrics0().getCacheHits();
++    }
++
++    /** {@inheritDoc} */
++    @Override public float getCacheHitPercentage() {
++        return cache.metrics0().getCacheHitPercentage();
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getCacheMisses() {
++        return cache.metrics0().getCacheMisses();
++    }
++
++    /** {@inheritDoc} */
++    @Override public float getCacheMissPercentage() {
++        return cache.metrics0().getCacheMissPercentage();
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getCacheGets() {
++        return cache.metrics0().getCacheGets();
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getCachePuts() {
++        return cache.metrics0().getCachePuts();
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getCacheRemovals() {
++        return cache.metrics0().getCacheRemovals();
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getCacheEvictions() {
++        return cache.metrics0().getCacheEvictions();
++    }
++
++    /** {@inheritDoc} */
++    @Override public float getAverageGetTime() {
++        return cache.metrics0().getAverageGetTime();
++    }
++
++    /** {@inheritDoc} */
++    @Override public float getAveragePutTime() {
++        return cache.metrics0().getAveragePutTime();
++    }
++
++    /** {@inheritDoc} */
++    @Override public float getAverageRemoveTime() {
++        return cache.metrics0().getAverageRemoveTime();
++    }
++
++    /** {@inheritDoc} */
++    @Override public float getAverageTxCommitTime() {
++        return cache.metrics0().getAverageTxCommitTime();
++    }
++
++    /** {@inheritDoc} */
++    @Override public float getAverageTxRollbackTime() {
++        return cache.metrics0().getAverageTxRollbackTime();
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getCacheTxCommits() {
++        return cache.metrics0().getCacheTxCommits();
++    }
++
++    /** {@inheritDoc} */
++    @Override public long getCacheTxRollbacks() {
++        return cache.metrics0().getCacheTxRollbacks();
++    }
++}

Reply via email to