# ignite-876
Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/68d5bd8a Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/68d5bd8a Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/68d5bd8a Branch: refs/heads/ignite-gg-10299 Commit: 68d5bd8a6ae31af1870b2e8d83eb070e8774e8e3 Parents: 5c30f9c Author: sboikov <sboi...@gridgain.com> Authored: Mon Jun 1 12:51:06 2015 +0300 Committer: sboikov <sboi...@gridgain.com> Committed: Mon Jun 1 17:28:45 2015 +0300 ---------------------------------------------------------------------- .../processors/cache/GridCacheMapEntry.java | 62 +++++- .../cache/distributed/dht/GridDhtGetFuture.java | 11 +- .../cache/GridCacheAbstractFullApiSelfTest.java | 27 +++ .../GridCachePartitionedFullApiSelfTest.java | 32 +++ ...edOffHeapTieredMultiNodeFullApiSelfTest.java | 2 +- .../GridCacheOffheapIndexEntryEvictTest.java | 200 +++++++++++++++++++ .../cache/GridCacheOffheapIndexGetSelfTest.java | 18 +- .../IgniteCacheWithIndexingTestSuite.java | 1 + 8 files changed, 337 insertions(+), 16 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/68d5bd8a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java index 92035af..0743aaa 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java @@ -61,6 +61,9 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx { private static final byte IS_UNSWAPPED_MASK = 0x02; /** */ + private static final byte IS_OFFHEAP_PTR_MASK = 0x04; + + /** */ public static final GridCacheAtomicVersionComparator ATOMIC_VER_COMPARATOR = new GridCacheAtomicVersionComparator(); /** @@ -433,6 +436,8 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx { if (e.offheapPointer() > 0) { offHeapPointer(e.offheapPointer()); + flags |= IS_OFFHEAP_PTR_MASK; + if (needVal) { CacheObject val = cctx.fromOffheap(offHeapPointer(), false); @@ -498,7 +503,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx { return; } - if (val == null && cctx.offheapTiered() && hasOffHeapPointer()) { + if (cctx.offheapTiered() && hasOffHeapPointer()) { if (log.isDebugEnabled()) log.debug("Value did not change, skip write swap entry: " + this); @@ -509,10 +514,16 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx { } IgniteUuid valClsLdrId = null; + IgniteUuid keyClsLdrId = null; - if (val != null) { - valClsLdrId = cctx.deploy().getClassLoaderId( - val.value(cctx.cacheObjectContext(), false).getClass().getClassLoader()); + if (cctx.kernalContext().config().isPeerClassLoadingEnabled()) { + if (val != null) { + valClsLdrId = cctx.deploy().getClassLoaderId( + U.detectObjectClassLoader(val.value(cctx.cacheObjectContext(), false))); + } + + keyClsLdrId = cctx.deploy().getClassLoaderId( + U.detectObjectClassLoader(key.value(cctx.cacheObjectContext(), false))); } IgniteBiTuple<byte[], Byte> valBytes = valueBytes0(); @@ -523,7 +534,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx { ver, ttlExtras(), expireTime, - cctx.deploy().getClassLoaderId(U.detectObjectClassLoader(key.value(cctx.cacheObjectContext(), false))), + keyClsLdrId, valClsLdrId); if (log.isDebugEnabled()) @@ -3617,6 +3628,8 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx { return true; } + else + evictFailed(prev); } } else { @@ -3660,8 +3673,11 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx { return true; } - else + else { + evictFailed(prevVal); + return false; + } } } } @@ -3680,6 +3696,25 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx { return false; } + /** + * @param prevVal Previous value. + * @throws IgniteCheckedException If failed. + */ + private void evictFailed(CacheObject prevVal) throws IgniteCheckedException { + if (cctx.offheapTiered() && ((flags & IS_OFFHEAP_PTR_MASK) != 0)) { + cctx.swap().removeOffheap(key()); + + offHeapPointer(0); + + flags &= ~IS_OFFHEAP_PTR_MASK; + + GridCacheQueryManager qryMgr = cctx.queries(); + + if (qryMgr != null) + qryMgr.onUnswap(key, prevVal); + } + } + /** {@inheritDoc} */ @Override public GridCacheBatchSwapEntry evictInBatchInternal(GridCacheVersion obsoleteVer) throws IgniteCheckedException { @@ -3692,10 +3727,17 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx { if (!hasReaders() && markObsolete0(obsoleteVer, false)) { if (!isStartVersion() && hasValueUnlocked()) { IgniteUuid valClsLdrId = null; + IgniteUuid keyClsLdrId = null; - if (val != null) - valClsLdrId = cctx.deploy().getClassLoaderId( - U.detectObjectClassLoader(val.value(cctx.cacheObjectContext(), false))); + if (cctx.kernalContext().config().isPeerClassLoadingEnabled()) { + if (val != null) { + valClsLdrId = cctx.deploy().getClassLoaderId( + U.detectObjectClassLoader(val.value(cctx.cacheObjectContext(), false))); + } + + keyClsLdrId = cctx.deploy().getClassLoaderId( + U.detectObjectClassLoader(key.value(cctx.cacheObjectContext(), false))); + } IgniteBiTuple<byte[], Byte> valBytes = valueBytes0(); @@ -3706,7 +3748,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx { ver, ttlExtras(), expireTimeExtras(), - cctx.deploy().getClassLoaderId(U.detectObjectClassLoader(key.value(cctx.cacheObjectContext(), false))), + keyClsLdrId, valClsLdrId); } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/68d5bd8a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java index f6f930e..742fbfe 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java @@ -295,6 +295,11 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col if (info == null) continue; + boolean addReader = (!e.deleted() && k.getValue() && !skipVals); + + if (addReader) + e.unswap(false); + // Register reader. If there are active transactions for this entry, // then will wait for their completion before proceeding. // TODO: GG-4003: @@ -303,8 +308,7 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col // TODO: To fix, check that reader is contained in the list of readers once // TODO: again after the returned future completes - if not, try again. // TODO: Also, why is info read before transactions are complete, and not after? - IgniteInternalFuture<Boolean> f = (!e.deleted() && k.getValue() && !skipVals) ? - e.addReader(reader, msgId, topVer) : null; + IgniteInternalFuture<Boolean> f = addReader ? e.addReader(reader, msgId, topVer) : null; if (f != null) { if (txFut == null) @@ -317,6 +321,9 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col break; } + catch (IgniteCheckedException err) { + return new GridFinishedFuture<>(err); + } catch (GridCacheEntryRemovedException ignore) { if (log.isDebugEnabled()) log.debug("Got removed entry when getting a DHT value: " + e); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/68d5bd8a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java index 70d8f9c..25f31ae 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java @@ -3913,6 +3913,33 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract /** * @throws Exception If failed. */ + public void testIterator() throws Exception { + IgniteCache<Integer, Integer> cache = grid(0).cache(null); + + final int KEYS = 1000; + + for (int i = 0; i < KEYS; i++) + cache.put(i, i); + + // Try to initialize readers in case when near cache is enabled. + for (int i = 0; i < gridCount(); i++) { + cache = grid(i).cache(null); + + for (int k = 0; k < KEYS; k++) + assertEquals((Object)k, cache.get(k)); + } + + int cnt = 0; + + for (Cache.Entry e : cache) + cnt++; + + assertEquals(KEYS, cnt); + } + + /** + * @throws Exception If failed. + */ public void testIgniteCacheIterator() throws Exception { IgniteCache<String, Integer> cache = jcache(0); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/68d5bd8a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java index 7e16ac7..da5d1bb 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java @@ -17,12 +17,15 @@ package org.apache.ignite.internal.processors.cache.distributed.near; +import org.apache.ignite.*; import org.apache.ignite.cache.*; import org.apache.ignite.cache.affinity.*; import org.apache.ignite.configuration.*; import org.apache.ignite.internal.*; import org.apache.ignite.internal.processors.cache.*; +import javax.cache.*; + import static org.apache.ignite.cache.CacheMode.*; /** @@ -72,4 +75,33 @@ public class GridCachePartitionedFullApiSelfTest extends GridCacheAbstractFullAp for (int i = 0 ; i < aff.partitions(); i++) String.valueOf(cache.entrySet(i)); } + + /** + * @throws Exception If failed. + */ + public void testUpdate() throws Exception { + if (gridCount() > 1) { + IgniteCache<Object, Object> cache = grid(0).cache(null); + + Integer key = nearKey(cache); + + primaryCache(key, null).put(key, 1); + + assertEquals(1, cache.get(key)); + + primaryCache(key, null).put(key, 2); + + if (cache.getConfiguration(CacheConfiguration.class).getNearConfiguration() != null) + assertEquals(2, cache.localPeek(key)); + + assertEquals(2, cache.get(key)); + + int cnt = 0; + + for (Cache.Entry e : cache) + cnt++; + + assertEquals(1, cnt); + } + } } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/68d5bd8a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOffHeapTieredMultiNodeFullApiSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOffHeapTieredMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOffHeapTieredMultiNodeFullApiSelfTest.java index 4392365..61c3563 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOffHeapTieredMultiNodeFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOffHeapTieredMultiNodeFullApiSelfTest.java @@ -65,7 +65,7 @@ public class GridCachePartitionedOffHeapTieredMultiNodeFullApiSelfTest extends G cache.put(key, 5); // Put from near to add reader on primary. assertEquals(5, primaryCache.localPeek(key, CachePeekMode.ONHEAP).intValue()); - assertEquals(5, primaryCache.localPeek(key, CachePeekMode.OFFHEAP).intValue()); + assertNull(primaryCache.localPeek(key, CachePeekMode.OFFHEAP)); assertEquals(5, cache.get(key).intValue()); assertEquals(5, map.get(key)); } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/68d5bd8a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexEntryEvictTest.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexEntryEvictTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexEntryEvictTest.java new file mode 100644 index 0000000..8afd746 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexEntryEvictTest.java @@ -0,0 +1,200 @@ +/* + * 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.query.*; +import org.apache.ignite.configuration.*; +import org.apache.ignite.spi.discovery.tcp.*; +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 javax.cache.*; +import java.io.*; +import java.util.*; +import java.util.concurrent.locks.*; + +import static org.apache.ignite.cache.CacheAtomicityMode.*; +import static org.apache.ignite.cache.CacheMemoryMode.*; +import static org.apache.ignite.cache.CacheMode.*; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*; + +/** + * + */ +public class GridCacheOffheapIndexEntryEvictTest extends GridCommonAbstractTest { + /** */ + private final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + TcpDiscoverySpi disco = new TcpDiscoverySpi(); + + disco.setIpFinder(ipFinder); + + cfg.setDiscoverySpi(disco); + + cfg.setNetworkTimeout(2000); + + cfg.setSwapSpaceSpi(new FileSwapSpaceSpi()); + + CacheConfiguration cacheCfg = defaultCacheConfiguration(); + + cacheCfg.setWriteSynchronizationMode(FULL_SYNC); + cacheCfg.setCacheMode(PARTITIONED); + cacheCfg.setBackups(1); + cacheCfg.setOffHeapMaxMemory(0); + cacheCfg.setAtomicityMode(TRANSACTIONAL); + cacheCfg.setMemoryMode(OFFHEAP_TIERED); + cacheCfg.setEvictionPolicy(null); + cacheCfg.setSqlOnheapRowCacheSize(10); + cacheCfg.setIndexedTypes(Integer.class, TestValue.class); + cacheCfg.setNearConfiguration(null); + + cfg.setCacheConfiguration(cacheCfg); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + startGrids(1); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testQueryWhenLocked() throws Exception { + IgniteCache<Integer, TestValue> cache = grid(0).cache(null); + + List<Lock> locks = new ArrayList<>(); + + final int ENTRIES = 1000; + + try { + for (int i = 0; i < ENTRIES; i++) { + cache.put(i, new TestValue(i)); + + Lock lock = cache.lock(i); + + lock.lock(); // Lock entry so that it should not be evicted. + + locks.add(lock); + + for (int j = 0; j < 3; j++) + assertNotNull(cache.get(i)); + } + + checkQuery(cache, "_key >= 100", ENTRIES - 100); + } + finally { + for (Lock lock : locks) + lock.unlock(); + } + } + + /** + * @throws Exception If failed. + */ + public void testUpdates() throws Exception { + final int ENTRIES = 500; + + IgniteCache<Integer, TestValue> cache = grid(0).cache(null); + + for (int i = 0; i < ENTRIES; i++) { + for (int j = 0; j < 3; j++) { + cache.getAndPut(i, new TestValue(i)); + + assertNotNull(cache.get(i)); + + assertNotNull(cache.localPeek(i)); + } + + checkQuery(cache, "_key >= 0", i + 1); + } + + for (int i = 0; i < ENTRIES; i++) { + if (i % 2 == 0) + cache.getAndRemove(i); + else + cache.remove(i); + + checkQuery(cache, "_key >= 0", ENTRIES - (i + 1)); + } + } + + /** + * @param cache Cache. + * @param sql Query. + * @param expCnt Number of expected entries. + */ + private void checkQuery(IgniteCache<Integer, TestValue> cache, String sql, int expCnt) { + SqlQuery<Integer, TestValue> qry = new SqlQuery<>(TestValue.class, sql); + + List<Cache.Entry<Integer, TestValue>> res = cache.query(qry).getAll(); + + assertEquals(expCnt, res.size()); + + for (Cache.Entry<Integer, TestValue> e : res) { + assertNotNull(e.getKey()); + + assertEquals((int)e.getKey(), e.getValue().val); + } + } + + /** + * + */ + static class TestValue implements Externalizable { + /** */ + private int val; + + /** + * + */ + public TestValue() { + // No-op. + } + + /** + * @param val Value. + */ + public TestValue(int val) { + this.val = val; + } + + /** {@inheritDoc} */ + @Override public void writeExternal(ObjectOutput out) throws IOException { + out.writeInt(val); + } + + /** {@inheritDoc} */ + @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + val = in.readInt(); + } + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/68d5bd8a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexGetSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexGetSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexGetSelfTest.java index 4e40040..41eb45a 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexGetSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexGetSelfTest.java @@ -18,6 +18,7 @@ package org.apache.ignite.internal.processors.cache; import org.apache.ignite.*; +import org.apache.ignite.cache.query.*; import org.apache.ignite.configuration.*; import org.apache.ignite.spi.discovery.tcp.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.*; @@ -25,6 +26,9 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*; import org.apache.ignite.spi.swapspace.file.*; import org.apache.ignite.testframework.junits.common.*; +import javax.cache.*; +import java.util.*; + import static org.apache.ignite.cache.CacheAtomicityMode.*; import static org.apache.ignite.cache.CacheMemoryMode.*; import static org.apache.ignite.cache.CacheMode.*; @@ -67,7 +71,6 @@ public class GridCacheOffheapIndexGetSelfTest extends GridCommonAbstractTest { cacheCfg.setAtomicityMode(TRANSACTIONAL); cacheCfg.setMemoryMode(OFFHEAP_TIERED); cacheCfg.setEvictionPolicy(null); - cacheCfg.setOffHeapMaxMemory(OFFHEAP_MEM); cacheCfg.setIndexedTypes(Long.class, Long.class); cfg.setCacheConfiguration(cacheCfg); @@ -98,8 +101,6 @@ public class GridCacheOffheapIndexGetSelfTest extends GridCommonAbstractTest { * @throws Exception If failed. */ public void testGet() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-873"); - IgniteCache<Long, Long> cache = grid(0).cache(null); for (long i = 0; i < 100; i++) @@ -107,5 +108,16 @@ public class GridCacheOffheapIndexGetSelfTest extends GridCommonAbstractTest { for (long i = 0; i < 100; i++) assertEquals((Long)i, cache.get(i)); + + SqlQuery<Long, Long> qry = new SqlQuery<>(Long.class, "_val >= 90"); + + List<Cache.Entry<Long, Long>> res = cache.query(qry).getAll(); + + assertEquals(10, res.size()); + + for (Cache.Entry<Long, Long> e : res) { + assertNotNull(e.getKey()); + assertNotNull(e.getValue()); + } } } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/68d5bd8a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java index ae45120..240caff 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java @@ -47,6 +47,7 @@ public class IgniteCacheWithIndexingTestSuite extends TestSuite { suite.addTestSuite(CacheTtlOnheapAtomicPartitionedSelfTest.class); suite.addTestSuite(GridCacheOffheapIndexGetSelfTest.class); + suite.addTestSuite(GridCacheOffheapIndexEntryEvictTest.class); suite.addTestSuite(CacheConfigurationP2PTest.class);