You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ak...@apache.org on 2016/03/30 08:46:48 UTC

ignite git commit: IGNITE-2910 Added detail info about keys count in partitions for offheap and swap.

Repository: ignite
Updated Branches:
  refs/heads/master 6a3d72480 -> 7081c0197


IGNITE-2910 Added detail info about keys count in partitions for offheap and swap.


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

Branch: refs/heads/master
Commit: 7081c0197ea146700f31dd222dba73d0a7428e5d
Parents: 6a3d724
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Wed Mar 30 13:46:37 2016 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Wed Mar 30 13:46:37 2016 +0700

----------------------------------------------------------------------
 .../processors/cache/GridCacheSwapManager.java  |  26 ++++-
 .../internal/visor/cache/VisorCacheV3.java      | 108 +++++++++++++++++++
 .../visor/node/VisorNodeDataCollectorJob.java   |  31 +++---
 3 files changed, 152 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7081c019/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
index 37c7958..d50bf0b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
@@ -306,6 +306,18 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
     }
 
     /**
+     * @param partId Partition ID to get swap entries count for.
+     * @return Number of swap entries.
+     * @throws IgniteCheckedException If failed.
+     */
+    public long swapEntriesCount(int partId) throws IgniteCheckedException {
+        if (!swapEnabled)
+            return 0;
+
+        return swapMgr.swapKeys(spaceName, Collections.singleton(partId));
+    }
+
+    /**
      * @param primary If {@code true} includes primary entries.
      * @param backup If {@code true} includes backup entries.
      * @param topVer Topology version.
@@ -329,6 +341,18 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
     }
 
     /**
+     * @param partId Partition ID to get entries count for.
+     * @return Number of offheap entries.
+     * @throws IgniteCheckedException If failed.
+     */
+    public long offheapEntriesCount(int partId) throws IgniteCheckedException {
+        if (!offheapEnabled)
+            return 0;
+
+        return offheap.entriesCount(spaceName, Collections.singleton(partId));
+    }
+
+    /**
      * Gets number of swap entries (keys).
      *
      * @return Swap keys count.
@@ -2538,4 +2562,4 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
             throw new UnsupportedOperationException();
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7081c019/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheV3.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheV3.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheV3.java
new file mode 100644
index 0000000..bd9a3ce
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheV3.java
@@ -0,0 +1,108 @@
+/*
+ * 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.visor.cache;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
+import org.apache.ignite.internal.processors.cache.GridCacheSwapManager;
+import org.apache.ignite.internal.util.lang.GridTuple3;
+import org.apache.ignite.internal.util.lang.IgnitePair;
+
+/**
+ * Data transfer object for {@link IgniteCache}.
+ */
+public class VisorCacheV3 extends VisorCacheV2 {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Primary partitions IDs with offheap and swap entries count. */
+    private Collection<GridTuple3<Integer, Long, Long>> primaryPartsOffheapSwap;
+
+    /** Backup partitions IDs with offheap and swap entries count. */
+    private Collection<GridTuple3<Integer, Long, Long>> backupPartsOffheapSwap;
+
+    /** {@inheritDoc} */
+    @Override public VisorCache from(IgniteEx ignite, String cacheName, int sample) throws IgniteCheckedException {
+        VisorCache c = super.from(ignite, cacheName, sample);
+
+        if (c != null && c instanceof VisorCacheV3) {
+            VisorCacheV3 cacheV3 = (VisorCacheV3)c;
+
+            GridCacheAdapter ca = ignite.context().cache().internalCache(cacheName);
+
+            // Process only started caches.
+            if (ca != null && ca.context().started()) {
+                GridCacheSwapManager swap = ca.context().swap();
+
+                cacheV3.primaryPartsOffheapSwap = new ArrayList<>(c.primaryPartitions().size());
+
+                for (IgnitePair<Integer> part: c.primaryPartitions()) {
+                    int p = part.get1();
+
+                    cacheV3.primaryPartsOffheapSwap.add(new GridTuple3<>(p, swap.offheapEntriesCount(p), swap.swapEntriesCount(p)));
+                }
+
+                cacheV3.backupPartsOffheapSwap = new ArrayList<>(c.backupPartitions().size());
+
+                for (IgnitePair<Integer> part: c.backupPartitions()) {
+                    int p = part.get1();
+
+                    cacheV3.backupPartsOffheapSwap.add(new GridTuple3<>(p, swap.offheapEntriesCount(p), swap.swapEntriesCount(p)));
+                }
+            }
+        }
+
+        return c;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected VisorCache initHistory(VisorCache c) {
+        super.initHistory(c);
+
+        if (c instanceof VisorCacheV3) {
+            ((VisorCacheV3)c).primaryPartsOffheapSwap = Collections.emptyList();
+            ((VisorCacheV3)c).backupPartsOffheapSwap = Collections.emptyList();
+        }
+
+        return c;
+    }
+
+    /** {@inheritDoc} */
+    @Override public VisorCache history() {
+        return initHistory(new VisorCacheV3());
+    }
+
+    /**
+     * @return Collection with primary partitions IDs and offheap and swap entries count.
+     */
+    public Collection<GridTuple3<Integer, Long, Long>> primaryPartitionsOffheapSwap() {
+        return primaryPartsOffheapSwap;
+    }
+
+    /**
+     * @return Collection with backup partitions IDs and offheap and swap entries count.
+     */
+    public Collection<GridTuple3<Integer, Long, Long>> backupPartitionsOffheapSwap() {
+        return backupPartsOffheapSwap;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7081c019/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
index f996d9a..79760ef 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
@@ -30,6 +30,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.visor.VisorJob;
 import org.apache.ignite.internal.visor.cache.VisorCache;
 import org.apache.ignite.internal.visor.cache.VisorCacheV2;
+import org.apache.ignite.internal.visor.cache.VisorCacheV3;
 import org.apache.ignite.internal.visor.compute.VisorComputeMonitoringHolder;
 import org.apache.ignite.internal.visor.igfs.VisorIgfs;
 import org.apache.ignite.internal.visor.igfs.VisorIgfsEndpoint;
@@ -53,6 +54,9 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
     /** */
     private static final IgniteProductVersion VER_1_4_1 = IgniteProductVersion.fromString("1.4.1");
 
+    /** */
+    private static final IgniteProductVersion VER_1_5_9 = IgniteProductVersion.fromString("1.5.9");
+
     /**
      * Create job with given argument.
      *
@@ -120,6 +124,18 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
     }
 
     /**
+     * @param ver Version to check.
+     * @return {@code true} if compatible.
+     */
+    private boolean compatibleWith(IgniteProductVersion ver) {
+        for (ClusterNode node : ignite.cluster().nodes())
+            if (node.version().compareToIgnoreTimestamp(ver) <= 0)
+                return true;
+
+        return false;
+    }
+
+    /**
      * Collect caches.
      *
      * @param res Job result.
@@ -136,18 +152,9 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
                     long start0 = U.currentTimeMillis();
 
                     try {
-                        boolean compatibility = false;
-
-                        for (ClusterNode node : ignite.cluster().nodes()) {
-                            if (node.version().compareToIgnoreTimestamp(VER_1_4_1) <= 0) {
-                                compatibility = true;
-
-                                break;
-                            }
-                        }
-
-                        VisorCache cache = (compatibility ? new VisorCache() : new VisorCacheV2())
-                                .from(ignite, cacheName, arg.sample());
+                        VisorCache cache = (compatibleWith(VER_1_4_1) ? new VisorCache() :
+                                compatibleWith(VER_1_5_9) ? new VisorCacheV2() : new VisorCacheV3())
+                                    .from(ignite, cacheName, arg.sample());
 
                         if (cache != null)
                             res.caches().add(cache);