Return-Path: X-Original-To: apmail-ignite-commits-archive@minotaur.apache.org Delivered-To: apmail-ignite-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 385921879C for ; Wed, 3 Jun 2015 17:15:03 +0000 (UTC) Received: (qmail 83472 invoked by uid 500); 3 Jun 2015 17:15:03 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 83440 invoked by uid 500); 3 Jun 2015 17:15:03 -0000 Mailing-List: contact commits-help@ignite.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.incubator.apache.org Delivered-To: mailing list commits@ignite.incubator.apache.org Received: (qmail 83431 invoked by uid 99); 3 Jun 2015 17:15:03 -0000 Received: from Unknown (HELO spamd2-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 03 Jun 2015 17:15:03 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd2-us-west.apache.org (ASF Mail Server at spamd2-us-west.apache.org) with ESMTP id 88CAC1A4464 for ; Wed, 3 Jun 2015 17:15:02 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 1.771 X-Spam-Level: * X-Spam-Status: No, score=1.771 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, KAM_LAZY_DOMAIN_SECURITY=1, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, T_RP_MATCHES_RCVD=-0.01, URIBL_BLOCKED=0.001] autolearn=disabled Received: from mx1-eu-west.apache.org ([10.40.0.8]) by localhost (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id VOT8o7i046pf for ; Wed, 3 Jun 2015 17:14:48 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-eu-west.apache.org (ASF Mail Server at mx1-eu-west.apache.org) with SMTP id CE84625497 for ; Wed, 3 Jun 2015 17:14:34 +0000 (UTC) Received: (qmail 80332 invoked by uid 99); 3 Jun 2015 17:14:33 -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, 03 Jun 2015 17:14:33 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id E68C4E098E; Wed, 3 Jun 2015 17:14:32 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: sboikov@apache.org To: commits@ignite.incubator.apache.org Date: Wed, 03 Jun 2015 17:14:43 -0000 Message-Id: <36a7b8d91060444ead7a16fdb1249f84@git.apache.org> In-Reply-To: <052e6ff62fa54a66be224a18e92c2d52@git.apache.org> References: <052e6ff62fa54a66be224a18e92c2d52@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [12/35] incubator-ignite git commit: ignite-916 Eviction policy should evict cache entries when memory size limit is reached ignite-916 Eviction policy should evict cache entries when memory size limit is reached Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/e5d5d08d Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/e5d5d08d Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/e5d5d08d Branch: refs/heads/ignite-218 Commit: e5d5d08d91fc0564b655c6e45afe0760957bff0c Parents: 5c30f9c Author: agura Authored: Mon May 18 21:01:36 2015 +0300 Committer: agura Committed: Fri May 29 21:12:32 2015 +0300 ---------------------------------------------------------------------- .../ignite/cache/eviction/EvictableEntry.java | 7 + .../ignite/cache/eviction/EvictionPolicy.java | 2 + .../cache/eviction/fifo/FifoEvictionPolicy.java | 117 +- .../eviction/fifo/FifoEvictionPolicyMBean.java | 22 + .../cache/eviction/lru/LruEvictionPolicy.java | 135 ++- .../eviction/lru/LruEvictionPolicyMBean.java | 38 + .../eviction/random/RandomEvictionPolicy.java | 7 +- .../eviction/sorted/SortedEvictionPolicy.java | 141 ++- .../sorted/SortedEvictionPolicyMBean.java | 22 + .../cache/CacheEvictableEntryImpl.java | 31 + .../GridCacheConcurrentTxMultiNodeTest.java | 8 +- ...idCacheConfigurationConsistencySelfTest.java | 14 +- .../cache/GridCacheMemoryModeSelfTest.java | 23 +- .../processors/cache/GridCacheOffHeapTest.java | 5 +- .../cache/GridCacheReloadSelfTest.java | 6 +- ...gniteCacheP2pUnmarshallingNearErrorTest.java | 6 +- .../cache/IgniteCachePeekModesAbstractTest.java | 5 +- ...GridCacheDhtEvictionNearReadersSelfTest.java | 11 +- .../dht/GridCacheDhtEvictionSelfTest.java | 11 +- .../dht/IgniteCacheMultiTxLockSelfTest.java | 6 +- .../GridCachePartitionedEvictionSelfTest.java | 11 +- ...ePartitionedMultiThreadedPutGetSelfTest.java | 6 +- .../cache/eviction/EvictionAbstractTest.java | 1057 ++++++++++++++++++ .../GridCacheBatchEvictUnswapSelfTest.java | 5 +- ...heConcurrentEvictionConsistencySelfTest.java | 82 +- .../GridCacheConcurrentEvictionsSelfTest.java | 29 +- .../GridCacheDistributedEvictionsSelfTest.java | 5 +- .../GridCacheEmptyEntriesAbstractSelfTest.java | 11 +- .../eviction/GridCacheEvictionAbstractTest.java | 484 -------- .../GridCacheEvictionTouchSelfTest.java | 22 +- .../cache/eviction/GridCacheMockEntry.java | 5 + .../fifo/FifoEvictionPolicySelfTest.java | 262 +++++ ...ridCacheFifoBatchEvictionPolicySelfTest.java | 384 ------- .../GridCacheFifoEvictionPolicySelfTest.java | 372 ------ .../lru/GridCacheLruEvictionPolicySelfTest.java | 417 ------- .../GridCacheLruNearEvictionPolicySelfTest.java | 136 --- ...heNearOnlyLruNearEvictionPolicySelfTest.java | 171 --- .../eviction/lru/LruEvictionPolicySelfTest.java | 353 ++++++ .../lru/LruNearEvictionPolicySelfTest.java | 140 +++ .../LruNearOnlyNearEvictionPolicySelfTest.java | 175 +++ .../GridCacheRandomEvictionPolicySelfTest.java | 258 ----- .../random/RandomEvictionPolicySelfTest.java | 357 ++++++ ...dCacheSortedBatchEvictionPolicySelfTest.java | 385 ------- ...acheSortedEvictionPolicyPerformanceTest.java | 135 --- .../GridCacheSortedEvictionPolicySelfTest.java | 373 ------ .../SortedEvictionPolicyPerformanceTest.java | 134 +++ .../sorted/SortedEvictionPolicySelfTest.java | 266 +++++ .../loadtests/GridCacheMultiNodeLoadTest.java | 5 +- .../GridCachePartitionedAtomicLongLoadTest.java | 6 +- .../swap/GridSwapEvictAllBenchmark.java | 6 +- .../IgniteCacheEvictionSelfTestSuite.java | 15 +- .../cache/GridIndexingWithNoopSwapSelfTest.java | 6 +- .../IgniteCacheQueryMultiThreadedSelfTest.java | 11 +- .../cache/ttl/CacheTtlAbstractSelfTest.java | 6 +- .../org/apache/ignite/yardstick/IgniteNode.java | 7 +- 55 files changed, 3480 insertions(+), 3234 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictableEntry.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictableEntry.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictableEntry.java index d87109f..9f1889a 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictableEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictableEntry.java @@ -46,6 +46,13 @@ public interface EvictableEntry extends Cache.Entry { public boolean isCached(); /** + * Returns entry size in bytes. + * + * @return entry size in bytes. + */ + public int size(); + + /** * Gets metadata added by eviction policy. * * @return Metadata value or {@code null}. http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictionPolicy.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictionPolicy.java index f409e9b..07c269d 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictionPolicy.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictionPolicy.java @@ -20,6 +20,7 @@ package org.apache.ignite.cache.eviction; import org.apache.ignite.cache.eviction.fifo.*; import org.apache.ignite.cache.eviction.lru.*; import org.apache.ignite.cache.eviction.random.*; +import org.apache.ignite.cache.eviction.sorted.*; /** * Pluggable cache eviction policy. Usually, implementations will internally order @@ -32,6 +33,7 @@ import org.apache.ignite.cache.eviction.random.*; *
  • {@link LruEvictionPolicy}
  • *
  • {@link RandomEvictionPolicy}
  • *
  • {@link FifoEvictionPolicy}
  • + *
  • {@link SortedEvictionPolicy}
  • * *

    * The eviction policy thread-safety is ensured by Ignition. Implementations of this interface should http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java index bf8cf0d..7222ae4 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java @@ -18,18 +18,28 @@ package org.apache.ignite.cache.eviction.fifo; import org.apache.ignite.cache.eviction.*; -import org.apache.ignite.configuration.*; import org.apache.ignite.internal.util.typedef.internal.*; + import org.jsr166.*; import org.jsr166.ConcurrentLinkedDeque8.*; import java.io.*; import java.util.*; +import static org.apache.ignite.configuration.CacheConfiguration.*; + /** * Eviction policy based on {@code First In First Out (FIFO)} algorithm and supports batch eviction. *

    - * The eviction starts when the cache size becomes {@code batchSize} elements greater than the maximum size. + * The eviction starts in the following cases: + *

      + *
    • The cache size becomes {@code batchSize} elements greater than the maximum size.
    • + *
    • + * The size of cache entries in bytes becomes greater than the maximum memory size. + * The size of cache entry calculates as sum of key size and value size. + *
    • + *
    + * Note:Batch eviction is enabled only if maximum memory limit isn't set ({@code maxMemSize == 0}). * {@code batchSize} elements will be evicted in this case. The default {@code batchSize} value is {@code 1}. *

    * This implementation is very efficient since it does not create any additional @@ -41,11 +51,17 @@ public class FifoEvictionPolicy implements EvictionPolicy, FifoEvict private static final long serialVersionUID = 0L; /** Maximum size. */ - private volatile int max = CacheConfiguration.DFLT_CACHE_SIZE; + private volatile int max = DFLT_CACHE_SIZE; /** Batch size. */ private volatile int batchSize = 1; + /** Max memory size. */ + private volatile long maxMemSize; + + /** Memory size. */ + private final LongAdder8 memSize = new LongAdder8(); + /** FIFO queue. */ private final ConcurrentLinkedDeque8> queue = new ConcurrentLinkedDeque8<>(); @@ -63,7 +79,7 @@ public class FifoEvictionPolicy implements EvictionPolicy, FifoEvict * @param max Maximum allowed size of cache before entry will start getting evicted. */ public FifoEvictionPolicy(int max) { - A.ensure(max > 0, "max > 0"); + A.ensure(max >= 0, "max >= 0"); this.max = max; } @@ -75,7 +91,7 @@ public class FifoEvictionPolicy implements EvictionPolicy, FifoEvict * @param batchSize Batch size. */ public FifoEvictionPolicy(int max, int batchSize) { - A.ensure(max > 0, "max > 0"); + A.ensure(max >= 0, "max >= 0"); A.ensure(batchSize > 0, "batchSize > 0"); this.max = max; @@ -97,7 +113,7 @@ public class FifoEvictionPolicy implements EvictionPolicy, FifoEvict * @param max Maximum allowed size of cache before entry will start getting evicted. */ @Override public void setMaxSize(int max) { - A.ensure(max > 0, "max > 0"); + A.ensure(max >= 0, "max >= 0"); this.max = max; } @@ -119,6 +135,23 @@ public class FifoEvictionPolicy implements EvictionPolicy, FifoEvict return queue.size(); } + /** {@inheritDoc} */ + @Override public long getMaxMemSize() { + return maxMemSize; + } + + /** {@inheritDoc} */ + @Override public void setMaxMemSize(long maxMemSize) { + A.ensure(maxMemSize >= 0, "maxMemSize >= 0"); + + this.maxMemSize = maxMemSize; + } + + /** {@inheritDoc} */ + @Override public long getCurrentMemSize() { + return memSize.longValue(); + } + /** * Gets read-only view on internal {@code FIFO} queue in proper order. * @@ -141,8 +174,11 @@ public class FifoEvictionPolicy implements EvictionPolicy, FifoEvict else { Node> node = entry.removeMeta(); - if (node != null) + if (node != null) { queue.unlinkx(node); + + memSize.add(-entry.size()); + } } } @@ -173,11 +209,18 @@ public class FifoEvictionPolicy implements EvictionPolicy, FifoEvict return false; } + memSize.add(entry.size()); + return true; } // If node was unlinked by concurrent shrink() call, we must repeat the whole cycle. else if (!entry.removeMeta(node)) return false; + else { + memSize.add(-entry.size()); + + return true; + } } } @@ -189,38 +232,74 @@ public class FifoEvictionPolicy implements EvictionPolicy, FifoEvict * Shrinks FIFO queue to maximum allowed size. */ private void shrink() { + long maxMem = this.maxMemSize; + + if (maxMem > 0) { + long startMemSize = memSize.longValue(); + + if (startMemSize >= maxMem) + for (long i = maxMem; i < startMemSize && memSize.longValue() > maxMem;) { + int size = shrink0(); + + if (size == -1) + break; + + i += size; + } + } + int max = this.max; - int batchSize = this.batchSize; + if (max > 0) { + int startSize = queue.sizex(); + + // Shrink only if queue is full. + if (startSize >= max + (maxMem > 0 ? 1 : this.batchSize)) + for (int i = max; i < startSize && queue.sizex() > max; i++) + if (shrink0() == -1) + break; + } + } - int startSize = queue.sizex(); + /** + * Tries to remove one item from queue. + * + * @return number of bytes that was free. {@code -1} if queue is empty. + */ + private int shrink0() { + EvictableEntry entry = queue.poll(); - // Shrink only if queue is full. - if (startSize >= max + batchSize) { - for (int i = max; i < startSize && queue.sizex() > max; i++) { - EvictableEntry entry = queue.poll(); + if (entry == null) + return -1; - if (entry == null) - break; + int size = 0; - Node> meta = entry.removeMeta(); + Node> meta = entry.removeMeta(); - if (meta != null && !entry.evict()) - touch(entry); - } + if (meta != null) { + size = entry.size(); + + memSize.add(-size); + + if (!entry.evict()) + touch(entry); } + + return size; } /** {@inheritDoc} */ @Override public void writeExternal(ObjectOutput out) throws IOException { out.writeInt(max); out.writeInt(batchSize); + out.writeLong(maxMemSize); } /** {@inheritDoc} */ @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { max = in.readInt(); batchSize = in.readInt(); + maxMemSize = in.readLong(); } /** {@inheritDoc} */ http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicyMBean.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicyMBean.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicyMBean.java index 63a413e..c9a09ec 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicyMBean.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicyMBean.java @@ -63,4 +63,26 @@ public interface FifoEvictionPolicyMBean { */ @MXBeanDescription("Current FIFO queue size.") public int getCurrentSize(); + + /** + * Gets maximum allowed cache size in bytes. + * + * @return maximum allowed cache size in bytes. + */ + @MXBeanDescription("Maximum allowed cache size in bytes.") + public long getMaxMemSize(); + + /** + * Sets maximum allowed cache size in bytes. + */ + @MXBeanDescription("Set maximum allowed cache size in bytes.") + public void setMaxMemSize(long maxMemSize); + + /** + * Gets current queue size in bytes. + * + * @return current queue size in bytes. + */ + @MXBeanDescription("Current FIFO queue size in bytes.") + public long getCurrentMemSize(); } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java index 309d577..d56f99a 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java @@ -18,26 +18,48 @@ package org.apache.ignite.cache.eviction.lru; import org.apache.ignite.cache.eviction.*; -import org.apache.ignite.configuration.*; import org.apache.ignite.internal.util.typedef.internal.*; + import org.jsr166.*; import org.jsr166.ConcurrentLinkedDeque8.*; import java.io.*; import java.util.*; +import static org.apache.ignite.configuration.CacheConfiguration.*; + /** - * Eviction policy based on {@code Least Recently Used (LRU)} algorithm. This - * implementation is very efficient since it is lock-free and does not - * create any additional table-like data structures. The {@code LRU} ordering - * information is maintained by attaching ordering metadata to cache entries. + * Eviction policy based on {@code Least Recently Used (LRU)} algorithm and supports batch eviction. + *

    + * The eviction starts in the following cases: + *

      + *
    • The cache size becomes {@code batchSize} elements greater than the maximum size.
    • + *
    • + * The size of cache entries in bytes becomes greater than the maximum memory size. + * The size of cache entry calculates as sum of key size and value size. + *
    • + *
    + * Note:Batch eviction is enabled only if maximum memory limit isn't set ({@code maxMemSize == 0}). + * {@code batchSize} elements will be evicted in this case. The default {@code batchSize} value is {@code 1}. + + * This implementation is very efficient since it is lock-free and does not create any additional table-like + * data structures. The {@code LRU} ordering information is maintained by attaching ordering metadata to cache entries. */ public class LruEvictionPolicy implements EvictionPolicy, LruEvictionPolicyMBean, Externalizable { /** */ private static final long serialVersionUID = 0L; /** Maximum size. */ - private volatile int max = CacheConfiguration.DFLT_CACHE_SIZE; + private volatile int max = DFLT_CACHE_SIZE; + + /** Batch size. */ + private volatile int batchSize = 1; + + /** Max memory size. */ + private volatile long maxMemSize; + + /** Memory size. */ + private final LongAdder8 memSize = new LongAdder8(); /** Queue. */ private final ConcurrentLinkedDeque8> queue = @@ -56,7 +78,7 @@ public class LruEvictionPolicy implements EvictionPolicy, LruEvictio * @param max Maximum allowed size of cache before entry will start getting evicted. */ public LruEvictionPolicy(int max) { - A.ensure(max > 0, "max > 0"); + A.ensure(max >= 0, "max >= 0"); this.max = max; } @@ -76,16 +98,45 @@ public class LruEvictionPolicy implements EvictionPolicy, LruEvictio * @param max Maximum allowed size of cache before entry will start getting evicted. */ @Override public void setMaxSize(int max) { - A.ensure(max > 0, "max > 0"); + A.ensure(max >= 0, "max >= 0"); this.max = max; } /** {@inheritDoc} */ + @Override public int getBatchSize() { + return batchSize; + } + + /** {@inheritDoc} */ + @Override public void setBatchSize(int batchSize) { + A.ensure(batchSize > 0, "batchSize > 0"); + + this.batchSize = batchSize; + } + + /** {@inheritDoc} */ @Override public int getCurrentSize() { return queue.size(); } + /** {@inheritDoc} */ + @Override public long getMaxMemSize() { + return maxMemSize; + } + + /** {@inheritDoc} */ + @Override public void setMaxMemSize(long maxMemSize) { + A.ensure(maxMemSize >= 0, "maxMemSize >= 0"); + + this.maxMemSize = maxMemSize; + } + + /** {@inheritDoc} */ + @Override public long getCurrentMemSize() { + return memSize.longValue(); + } + /** * Gets read-only view on internal {@code FIFO} queue in proper order. * @@ -107,8 +158,11 @@ public class LruEvictionPolicy implements EvictionPolicy, LruEvictio else { Node> node = entry.removeMeta(); - if (node != null) + if (node != null) { queue.unlinkx(node); + + memSize.add(-entry.size()); + } } } @@ -139,11 +193,18 @@ public class LruEvictionPolicy implements EvictionPolicy, LruEvictio return false; } + memSize.add(entry.size()); + return true; } // If node was unlinked by concurrent shrink() call, we must repeat the whole cycle. else if (!entry.removeMeta(node)) return false; + else { + memSize.add(-entry.size()); + + return true; + } } } else if (queue.unlinkx(node)) { @@ -163,31 +224,73 @@ public class LruEvictionPolicy implements EvictionPolicy, LruEvictio * Shrinks queue to maximum allowed size. */ private void shrink() { + long maxMem = this.maxMemSize; + + if (maxMem > 0) { + long startMemSize = memSize.longValue(); + + if (startMemSize >= maxMem) + for (long i = maxMem; i < startMemSize && memSize.longValue() > maxMem;) { + int size = shrink0(); + + if (size == -1) + break; + + i += size; + } + } + int max = this.max; - int startSize = queue.sizex(); + if (max > 0) { + int startSize = queue.sizex(); - for (int i = 0; i < startSize && queue.sizex() > max; i++) { - EvictableEntry entry = queue.poll(); + if (startSize >= max + (maxMem > 0 ? 1 : this.batchSize)) + for (int i = max; i < startSize && queue.sizex() > max; i++) + if (shrink0() == -1) + break; + } + } - if (entry == null) - break; + /** + * Tries to remove one item from queue. + * + * @return number of bytes that was free. {@code -1} if queue is empty. + */ + private int shrink0() { + EvictableEntry entry = queue.poll(); - Node> meta = entry.removeMeta(); + if (entry == null) + return -1; - if (meta != null && !entry.evict()) + int size = 0; + + Node> meta = entry.removeMeta(); + + if (meta != null) { + size = entry.size(); + + memSize.add(-size); + + if (!entry.evict()) touch(entry); } + + return size; } /** {@inheritDoc} */ @Override public void writeExternal(ObjectOutput out) throws IOException { out.writeInt(max); + out.writeInt(batchSize); + out.writeLong(maxMemSize); } /** {@inheritDoc} */ @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { max = in.readInt(); + batchSize = in.readInt(); + maxMemSize = in.readLong(); } /** {@inheritDoc} */ http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicyMBean.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicyMBean.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicyMBean.java index c243374..69347ee 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicyMBean.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicyMBean.java @@ -41,10 +41,48 @@ public interface LruEvictionPolicyMBean { public void setMaxSize(int max); /** + * Gets batch size. + * + * @return batch size. + */ + @MXBeanDescription("Batch size.") + public int getBatchSize(); + + /** + * Sets batch size. + * + * @param batchSize Batch size. + */ + @MXBeanDescription("Set batch size.") + public void setBatchSize(int batchSize); + + /** * Gets current queue size. * * @return Current queue size. */ @MXBeanDescription("Current queue size.") public int getCurrentSize(); + + /** + * Gets maximum allowed cache size in bytes. + * + * @return maximum allowed cache size in bytes. + */ + @MXBeanDescription("Maximum allowed cache size in bytes.") + public long getMaxMemSize(); + + /** + * Sets maximum allowed cache size in bytes. + */ + @MXBeanDescription("Set maximum allowed cache size in bytes.") + public void setMaxMemSize(long maxMemSize); + + /** + * Gets current queue size in bytes. + * + * @return current queue size in bytes. + */ + @MXBeanDescription("Current queue size in bytes.") + public long getCurrentMemSize(); } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicy.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicy.java index c88b31d..0d840e3 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicy.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicy.java @@ -19,19 +19,20 @@ package org.apache.ignite.cache.eviction.random; import org.apache.ignite.*; import org.apache.ignite.cache.eviction.*; -import org.apache.ignite.configuration.*; import org.apache.ignite.internal.util.typedef.internal.*; import javax.cache.*; import java.io.*; +import static org.apache.ignite.configuration.CacheConfiguration.*; + /** * Cache eviction policy which will select random cache entry for eviction if cache * size exceeds the {@link #getMaxSize()} parameter. This implementation is * extremely light weight, lock-free, and does not create any data structures to maintain * any order for eviction. *

    - * Random eviction will provide the best performance over any key set in which every + * Random eviction will provide the best performance over any key queue in which every * key has the same probability of being accessed. */ public class RandomEvictionPolicy implements EvictionPolicy, RandomEvictionPolicyMBean, Externalizable { @@ -39,7 +40,7 @@ public class RandomEvictionPolicy implements EvictionPolicy, RandomE private static final long serialVersionUID = 0L; /** Maximum size. */ - private volatile int max = CacheConfiguration.DFLT_CACHE_SIZE; + private volatile int max = DFLT_CACHE_SIZE; /** * Constructs random eviction policy with all defaults. http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java index 7965c97..abfc1d0 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java @@ -34,7 +34,15 @@ import static org.apache.ignite.configuration.CacheConfiguration.*; /** * Cache eviction policy which will select the minimum cache entry for eviction. *

    - * The eviction starts when the cache size becomes {@code batchSize} elements greater than the maximum size. + * The eviction starts in the following cases: + *

      + *
    • The cache size becomes {@code batchSize} elements greater than the maximum size.
    • + *
    • + * The size of cache entries in bytes becomes greater than the maximum memory size. + * The size of cache entry calculates as sum of key size and value size. + *
    • + *
    + * Note:Batch eviction is enabled only if maximum memory limit isn't set ({@code maxMemSize == 0}). * {@code batchSize} elements will be evicted in this case. The default {@code batchSize} value is {@code 1}. *

    * Entries comparison based on {@link Comparator} instance if provided. @@ -48,18 +56,24 @@ public class SortedEvictionPolicy implements EvictionPolicy, SortedE private static final long serialVersionUID = 0L; /** Maximum size. */ - private volatile int max; + private volatile int max = DFLT_CACHE_SIZE; /** Batch size. */ private volatile int batchSize = 1; + /** Max memory size. */ + private volatile long maxMemSize; + + /** Memory size. */ + private final LongAdder8 memSize = new LongAdder8(); + /** Comparator. */ private Comparator> comp; /** Order. */ private final AtomicLong orderCnt = new AtomicLong(); - /** Backed sorted set. */ + /** Backed sorted queue. */ private final GridConcurrentSkipListSetEx set; /** @@ -96,7 +110,7 @@ public class SortedEvictionPolicy implements EvictionPolicy, SortedE * @param comp Entries comparator. */ public SortedEvictionPolicy(int max, int batchSize, @Nullable Comparator> comp) { - A.ensure(max > 0, "max > 0"); + A.ensure(max >= 0, "max >= 0"); A.ensure(batchSize > 0, "batchSize > 0"); this.max = max; @@ -106,6 +120,16 @@ public class SortedEvictionPolicy implements EvictionPolicy, SortedE } /** + * Constructs sorted eviction policy with given maximum size and given entry comparator. + * + * @param comp Entries comparator. + */ + public SortedEvictionPolicy(@Nullable Comparator> comp) { + this.comp = comp == null ? new DefaultHolderComparator() : new HolderComparator<>(comp); + this.set = new GridConcurrentSkipListSetEx<>(this.comp); + } + + /** * Gets maximum allowed size of cache before entry will start getting evicted. * * @return Maximum allowed size of cache before entry will start getting evicted. @@ -120,7 +144,7 @@ public class SortedEvictionPolicy implements EvictionPolicy, SortedE * @param max Maximum allowed size of cache before entry will start getting evicted. */ @Override public void setMaxSize(int max) { - A.ensure(max > 0, "max > 0"); + A.ensure(max >= 0, "max >= 0"); this.max = max; } @@ -142,12 +166,29 @@ public class SortedEvictionPolicy implements EvictionPolicy, SortedE return set.sizex(); } + /** {@inheritDoc} */ + @Override public long getMaxMemSize() { + return maxMemSize; + } + + /** {@inheritDoc} */ + @Override public void setMaxMemSize(long maxMemSize) { + A.ensure(maxMemSize >= 0, "maxMemSize >= 0"); + + this.maxMemSize = maxMemSize; + } + + /** {@inheritDoc} */ + @Override public long getCurrentMemSize() { + return memSize.longValue(); + } + /** - * Gets read-only view of backed set in proper order. + * Gets read-only view of backed queue in proper order. * - * @return Read-only view of backed set. + * @return Read-only view of backed queue. */ - public Collection> set() { + public Collection> queue() { Set> cp = new LinkedHashSet<>(); for (Holder holder : set) @@ -168,19 +209,22 @@ public class SortedEvictionPolicy implements EvictionPolicy, SortedE else { Holder holder = entry.removeMeta(); - if (holder != null) + if (holder != null) { removeHolder(holder); + + memSize.add(-entry.size()); + } } } /** * @param entry Entry to touch. - * @return {@code True} if backed set has been changed by this call. + * @return {@code True} if backed queue has been changed by this call. */ private boolean touch(EvictableEntry entry) { Holder holder = entry.meta(); - // Entry has not been add yet to backed set.. + // Entry has not been add yet to backed queue.. if (holder == null) { while (true) { holder = new Holder<>(entry, orderCnt.incrementAndGet()); @@ -188,7 +232,7 @@ public class SortedEvictionPolicy implements EvictionPolicy, SortedE set.add(holder); if (entry.putMetaIfAbsent(holder) != null) { - // Was concurrently added, need to remove it from set. + // Was concurrently added, need to remove it from queue. removeHolder(holder); // Set has not been changed. @@ -196,17 +240,24 @@ public class SortedEvictionPolicy implements EvictionPolicy, SortedE } else if (holder.order > 0) { if (!entry.isCached()) { - // Was concurrently evicted, need to remove it from set. + // Was concurrently evicted, need to remove it from queue. removeHolder(holder); return false; } + memSize.add(entry.size()); + return true; } // If holder was removed by concurrent shrink() call, we must repeat the whole cycle. else if (!entry.removeMeta(holder)) return false; + else { + memSize.add(-entry.size()); + + return true; + } } } @@ -215,34 +266,71 @@ public class SortedEvictionPolicy implements EvictionPolicy, SortedE } /** - * Shrinks backed set to maximum allowed size. + * Shrinks backed queue to maximum allowed size. */ private void shrink() { - int max = this.max; + long maxMem = this.maxMemSize; + + if (maxMem > 0) { + long startMemSize = memSize.longValue(); - int batchSize = this.batchSize; + if (startMemSize >= maxMem) + for (long i = maxMem; i < startMemSize && memSize.longValue() > maxMem;) { + int size = shrink0(); - int startSize = set.sizex(); + if (size == -1) + break; - if (startSize >= max + batchSize) { - for (int i = max; i < startSize && set.sizex() > max; i++) { - Holder h = set.pollFirst(); + i += size; + } + } - if (h == null) - break; + int max = this.max; - EvictableEntry entry = h.entry; + if (max > 0) { + int startSize = set.sizex(); - if (h.order > 0 && entry.removeMeta(h) && !entry.evict()) - touch(entry); + if (startSize >= max + (maxMem > 0 ? 1 : this.batchSize)) { + for (int i = max; i < startSize && set.sizex() > max; i++) { + if (shrink0() == -1) + break; + } } } } + /** + * Tries to remove one item from queue. + * + * @return number of bytes that was free. {@code -1} if queue is empty. + */ + private int shrink0() { + Holder h = set.pollFirst(); + + if (h == null) + return -1; + + int size = 0; + + EvictableEntry entry = h.entry; + + if (h.order > 0 && entry.removeMeta(h)) { + size = entry.size(); + + memSize.add(-size); + + if (!entry.evict()) + touch(entry); + } + + return size; + } + /** {@inheritDoc} */ @Override public void writeExternal(ObjectOutput out) throws IOException { out.writeInt(max); out.writeInt(batchSize); + out.writeLong(maxMemSize); out.writeObject(comp); } @@ -251,11 +339,12 @@ public class SortedEvictionPolicy implements EvictionPolicy, SortedE @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { max = in.readInt(); batchSize = in.readInt(); + maxMemSize = in.readLong(); comp = (Comparator>)in.readObject(); } /** - * Removes holder from backed set and marks holder as removed. + * Removes holder from backed queue and marks holder as removed. * * @param holder Holder. */ http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicyMBean.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicyMBean.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicyMBean.java index bc696ff..aada1ce 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicyMBean.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicyMBean.java @@ -63,4 +63,26 @@ public interface SortedEvictionPolicyMBean { */ @MXBeanDescription("Current sorted key set size.") public int getCurrentSize(); + + /** + * Gets maximum allowed cache size in bytes. + * + * @return maximum allowed cache size in bytes. + */ + @MXBeanDescription("Maximum allowed cache size in bytes.") + public long getMaxMemSize(); + + /** + * Sets maximum allowed cache size in bytes. + */ + @MXBeanDescription("Set maximum allowed cache size in bytes.") + public void setMaxMemSize(long maxMemSize); + + /** + * Gets current sorted entries queue size in bytes. + * + * @return current sorted entries queue size in bytes. + */ + @MXBeanDescription("Current sorted entries set size in bytes.") + public long getCurrentMemSize(); } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictableEntryImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictableEntryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictableEntryImpl.java index 5d6062e..7a3fbee 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictableEntryImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictableEntryImpl.java @@ -23,6 +23,7 @@ import org.apache.ignite.internal.processors.cache.transactions.*; import org.apache.ignite.internal.util.lang.*; import org.apache.ignite.internal.util.tostring.*; import org.apache.ignite.internal.util.typedef.internal.*; + import org.jetbrains.annotations.*; import java.util.*; @@ -91,6 +92,36 @@ public class CacheEvictableEntryImpl implements EvictableEntry { } /** {@inheritDoc} */ + public int size() { + try { + GridCacheContext cctx = cached.context(); + + KeyCacheObject key = cached.key(); + + byte[] keyBytes = key.valueBytes(cctx.cacheObjectContext()); + + byte[] valBytes = null; + + if (cctx.useOffheapEntry()) + valBytes = cctx.offheap().get(cctx.swap().spaceName(), cached.partition(), key, keyBytes); + else { + CacheObject cacheObj = cached.valueBytes(); + + if (cacheObj != null) + valBytes = cacheObj.valueBytes(cctx.cacheObjectContext()); + } + + return valBytes == null ? keyBytes.length : keyBytes.length + valBytes.length; + } + catch (GridCacheEntryRemovedException e) { + return 0; + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + } + + /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public V getValue() { try { http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java index 8a1ae78..bba4ad9 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java @@ -26,7 +26,6 @@ import org.apache.ignite.cluster.*; import org.apache.ignite.compute.*; import org.apache.ignite.configuration.*; import org.apache.ignite.internal.*; -import org.apache.ignite.internal.processors.cache.query.*; import org.apache.ignite.internal.processors.cache.distributed.dht.*; import org.apache.ignite.internal.processors.cache.distributed.near.*; import org.apache.ignite.internal.util.*; @@ -38,6 +37,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*; import org.apache.ignite.testframework.junits.common.*; import org.apache.ignite.transactions.*; + import org.jetbrains.annotations.*; import java.io.*; @@ -110,7 +110,11 @@ public class GridCacheConcurrentTxMultiNodeTest extends GridCommonAbstractTest { CacheConfiguration cc = defaultCacheConfiguration(); cc.setCacheMode(mode); - cc.setEvictionPolicy(new LruEvictionPolicy(1000)); + + LruEvictionPolicy plc = new LruEvictionPolicy(); + plc.setMaxSize(1000); + + cc.setEvictionPolicy(plc); cc.setEvictSynchronized(false); cc.setSwapEnabled(false); cc.setWriteSynchronizationMode(FULL_SYNC); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java index 44171a8..445a5e1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java @@ -442,7 +442,12 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac /** {@inheritDoc} */ @Override public Void apply(CacheConfiguration cfg) { cfg.setEvictSynchronized(true); - cfg.setEvictionPolicy(new FifoEvictionPolicy(100)); + + FifoEvictionPolicy plc = new FifoEvictionPolicy(); + + plc.setMaxSize(100); + + cfg.setEvictionPolicy(plc); return null; } }, @@ -450,7 +455,12 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac /** {@inheritDoc} */ @Override public Void apply(CacheConfiguration cfg) { cfg.setEvictSynchronized(false); - cfg.setEvictionPolicy(new FifoEvictionPolicy(100)); + + FifoEvictionPolicy plc = new FifoEvictionPolicy(); + + plc.setMaxSize(100); + + cfg.setEvictionPolicy(plc); return null; } } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java index 08e35ce..b4e523e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java @@ -28,6 +28,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*; import org.apache.ignite.spi.swapspace.file.*; import org.apache.ignite.testframework.junits.common.*; + import org.junit.*; import java.util.*; @@ -81,6 +82,11 @@ public class GridCacheMemoryModeSelfTest extends GridCommonAbstractTest { return cfg; } + /** + * Returns cache configuration. + * + * @return cache configuration. + */ protected CacheConfiguration cacheConfiguration() { CacheConfiguration cacheCfg = defaultCacheConfiguration(); @@ -89,7 +95,16 @@ public class GridCacheMemoryModeSelfTest extends GridCommonAbstractTest { cacheCfg.setSwapEnabled(swapEnabled); cacheCfg.setCacheMode(mode); cacheCfg.setMemoryMode(memoryMode); - cacheCfg.setEvictionPolicy(maxOnheapSize == Integer.MAX_VALUE ? null : new LruEvictionPolicy(maxOnheapSize)); + + LruEvictionPolicy plc = null; + + if (maxOnheapSize != Integer.MAX_VALUE) { + plc = new LruEvictionPolicy(); + plc.setMaxSize(maxOnheapSize); + } + + cacheCfg.setEvictionPolicy(plc); + cacheCfg.setAtomicityMode(atomicity); cacheCfg.setOffHeapMaxMemory(offheapSize); @@ -199,7 +214,8 @@ public class GridCacheMemoryModeSelfTest extends GridCommonAbstractTest { * @param swapEmpty Swap is empty. * @throws Exception If failed. */ - private void doTestPutAndPutAll(int cache, int offheapSwap, boolean offheapEmpty, boolean swapEmpty) throws Exception { + private void doTestPutAndPutAll(int cache, int offheapSwap, boolean offheapEmpty, boolean swapEmpty) + throws Exception { final int all = cache + offheapSwap; // put @@ -231,7 +247,8 @@ public class GridCacheMemoryModeSelfTest extends GridCommonAbstractTest { * @param x Cache modifier. * @throws IgniteCheckedException If failed. */ - void doTest(int cache, int offheapSwap, boolean offheapEmpty, boolean swapEmpty, CIX1> x) throws Exception { + void doTest(int cache, int offheapSwap, boolean offheapEmpty, boolean swapEmpty, + CIX1> x) throws Exception { ipFinder = new TcpDiscoveryVmIpFinder(true); startGrid(); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapTest.java index d69f7fa..b0f07f0 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapTest.java @@ -76,7 +76,10 @@ public class GridCacheOffHeapTest extends GridCommonAbstractTest { cacheCfg.setStartSize(startSize); if (onheap > 0) { - cacheCfg.setEvictionPolicy(new FifoEvictionPolicy(onheap)); + FifoEvictionPolicy plc = new FifoEvictionPolicy(); + plc.setMaxSize(onheap); + + cacheCfg.setEvictionPolicy(plc); cacheCfg.setOffHeapMaxMemory(80 * 1024L * 1024L * 1024L); // 10GB } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReloadSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReloadSelfTest.java index e2cdd08..b6bfbc7 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReloadSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReloadSelfTest.java @@ -74,7 +74,11 @@ public class GridCacheReloadSelfTest extends GridCommonAbstractTest { CacheConfiguration cacheCfg = defaultCacheConfiguration(); cacheCfg.setName(CACHE_NAME); cacheCfg.setCacheMode(cacheMode); - cacheCfg.setEvictionPolicy(new LruEvictionPolicy(MAX_CACHE_ENTRIES)); + + LruEvictionPolicy plc = new LruEvictionPolicy(); + plc.setMaxSize(MAX_CACHE_ENTRIES); + + cacheCfg.setEvictionPolicy(plc); cacheCfg.setNearConfiguration(nearEnabled ? new NearCacheConfiguration() : null); final CacheStore store = new CacheStoreAdapter() { http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java index 732d12d..30f7b92 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java @@ -36,7 +36,11 @@ public class IgniteCacheP2pUnmarshallingNearErrorTest extends IgniteCacheP2pUnma cfg.getCacheConfiguration()[0].setEvictMaxOverflowRatio(0); cfg.getCacheConfiguration()[0].setEvictSynchronized(true); cfg.getCacheConfiguration()[0].setEvictSynchronizedKeyBufferSize(1); - cfg.getCacheConfiguration()[0].setEvictionPolicy(new FifoEvictionPolicy(1)); + + FifoEvictionPolicy plc = new FifoEvictionPolicy(); + plc.setMaxSize(1); + + cfg.getCacheConfiguration()[0].setEvictionPolicy(plc); return cfg; } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java index 7cd8414..448f171 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java @@ -93,7 +93,10 @@ public abstract class IgniteCachePeekModesAbstractTest extends IgniteCacheAbstra if (hasNearCache()) ccfg.setNearConfiguration(new NearCacheConfiguration()); - ccfg.setEvictionPolicy(new FifoEvictionPolicy(HEAP_ENTRIES)); + FifoEvictionPolicy plc = new FifoEvictionPolicy(); + plc.setMaxSize(HEAP_ENTRIES); + + ccfg.setEvictionPolicy(plc); return ccfg; } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java index aff5512..7adeba8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java @@ -77,11 +77,18 @@ public class GridCacheDhtEvictionNearReadersSelfTest extends GridCommonAbstractT // Set eviction queue size explicitly. cacheCfg.setEvictSynchronizedKeyBufferSize(1); cacheCfg.setEvictMaxOverflowRatio(0); - cacheCfg.setEvictionPolicy(new FifoEvictionPolicy(10)); + + FifoEvictionPolicy plc = new FifoEvictionPolicy(); + plc.setMaxSize(10); + + cacheCfg.setEvictionPolicy(plc); NearCacheConfiguration nearCfg = new NearCacheConfiguration(); - nearCfg.setNearEvictionPolicy(new FifoEvictionPolicy(10)); + FifoEvictionPolicy nearPlc = new FifoEvictionPolicy(); + nearPlc.setMaxSize(10); + + nearCfg.setNearEvictionPolicy(nearPlc); cacheCfg.setNearConfiguration(nearCfg); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java index 97e8657..8eada9c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java @@ -76,14 +76,21 @@ public class GridCacheDhtEvictionSelfTest extends GridCommonAbstractTest { cacheCfg.setBackups(1); NearCacheConfiguration nearCfg = new NearCacheConfiguration(); - nearCfg.setNearEvictionPolicy(new FifoEvictionPolicy(10000)); + + FifoEvictionPolicy nearPlc = new FifoEvictionPolicy(); + nearPlc.setMaxSize(10000); + + nearCfg.setNearEvictionPolicy(nearPlc); cacheCfg.setNearConfiguration(nearCfg); // Set eviction queue size explicitly. cacheCfg.setEvictMaxOverflowRatio(0); cacheCfg.setEvictSynchronizedKeyBufferSize(1); - cacheCfg.setEvictionPolicy(new FifoEvictionPolicy(10000)); + FifoEvictionPolicy plc = new FifoEvictionPolicy(); + plc.setMaxSize(10000); + + cacheCfg.setEvictionPolicy(plc); cfg.setCacheConfiguration(cacheCfg); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java index 5983c1b..77fe8cf 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java @@ -71,7 +71,11 @@ public class IgniteCacheMultiTxLockSelfTest extends GridCommonAbstractTest { ccfg.setBackups(2); ccfg.setCacheMode(CacheMode.PARTITIONED); ccfg.setStartSize(100000); - ccfg.setEvictionPolicy(new LruEvictionPolicy(100000)); + + LruEvictionPolicy plc = new LruEvictionPolicy(); + plc.setMaxSize(100000); + + ccfg.setEvictionPolicy(plc); ccfg.setEvictSynchronized(true); c.setCacheConfiguration(ccfg); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEvictionSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEvictionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEvictionSelfTest.java index 28ad7a8..c0d36e2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEvictionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEvictionSelfTest.java @@ -78,8 +78,15 @@ public class GridCachePartitionedEvictionSelfTest extends GridCacheAbstractSelfT cc.setCacheMode(PARTITIONED); cc.setWriteSynchronizationMode(FULL_SYNC); - cc.setEvictionPolicy(new FifoEvictionPolicy(EVICT_CACHE_SIZE)); - cc.getNearConfiguration().setNearEvictionPolicy(new FifoEvictionPolicy(EVICT_CACHE_SIZE)); + + FifoEvictionPolicy plc = new FifoEvictionPolicy(); + plc.setMaxSize(EVICT_CACHE_SIZE); + cc.setEvictionPolicy(plc); + + FifoEvictionPolicy nearPlc = new FifoEvictionPolicy(); + nearPlc.setMaxSize(EVICT_CACHE_SIZE); + cc.getNearConfiguration().setNearEvictionPolicy(nearPlc); + cc.setSwapEnabled(false); // We set 1 backup explicitly. http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiThreadedPutGetSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiThreadedPutGetSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiThreadedPutGetSelfTest.java index 8131d3e..1c45de8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiThreadedPutGetSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiThreadedPutGetSelfTest.java @@ -66,7 +66,11 @@ public class GridCachePartitionedMultiThreadedPutGetSelfTest extends GridCommonA cc.setCacheMode(PARTITIONED); cc.setBackups(1); cc.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); - cc.setEvictionPolicy(new FifoEvictionPolicy<>(1000)); + + FifoEvictionPolicy plc = new FifoEvictionPolicy(); + plc.setMaxMemSize(1000); + + cc.setEvictionPolicy(plc); cc.setSwapEnabled(false); cc.setAtomicityMode(TRANSACTIONAL); cc.setEvictSynchronized(false);