# ignite-57
Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/d4ddf85c Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/d4ddf85c Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/d4ddf85c Branch: refs/heads/ignite-96 Commit: d4ddf85c95b1dc121d943494ef3dad3a6b5b849d Parents: d58e808 Author: sboikov <semen.boi...@inria.fr> Authored: Mon Feb 9 23:11:28 2015 +0300 Committer: sboikov <semen.boi...@inria.fr> Committed: Mon Feb 9 23:11:28 2015 +0300 ---------------------------------------------------------------------- .../processors/cache/GridCacheAdapter.java | 122 +++---- .../processors/cache/GridCacheSwapManager.java | 2 +- .../cache/IgniteCachePeekAbstractTest.java | 333 ------------------- 3 files changed, 46 insertions(+), 411 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d4ddf85c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java index fdee08e..55c9445 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java @@ -1296,22 +1296,22 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, /** {@inheritDoc} */ @Override public Set<CacheEntry<K, V>> primaryEntrySet() { - return primaryEntrySet((IgnitePredicate<CacheEntry<K, V>>[]) null); + return primaryEntrySet((IgnitePredicate<CacheEntry<K, V>>[])null); } /** {@inheritDoc} */ @Override public Set<K> keySet() { - return keySet((IgnitePredicate<CacheEntry<K, V>>[]) null); + return keySet((IgnitePredicate<CacheEntry<K, V>>[])null); } /** {@inheritDoc} */ @Override public Set<K> primaryKeySet() { - return primaryKeySet((IgnitePredicate<CacheEntry<K, V>>[]) null); + return primaryKeySet((IgnitePredicate<CacheEntry<K, V>>[])null); } /** {@inheritDoc} */ @Override public Collection<V> values() { - return values((IgnitePredicate<CacheEntry<K, V>>[]) null); + return values((IgnitePredicate<CacheEntry<K, V>>[])null); } /** {@inheritDoc} */ @@ -1611,8 +1611,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, return getAllAsync(Collections.singletonList(key), /*force primary*/true, /*skip tx*/false, null, null, taskName, true).chain(new CX1<IgniteInternalFuture<Map<K, V>>, V>() { - @Override - public V applyx(IgniteInternalFuture<Map<K, V>> e) throws IgniteCheckedException { + @Override public V applyx(IgniteInternalFuture<Map<K, V>> e) throws IgniteCheckedException { return e.get().get(key); } }); @@ -1634,7 +1633,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, /** {@inheritDoc} */ @Nullable @Override public V reload(K key) throws IgniteCheckedException { - return reload(key, (IgnitePredicate<CacheEntry<K, V>>[]) null); + return reload(key, (IgnitePredicate<CacheEntry<K, V>>[])null); } /** {@inheritDoc} */ @@ -1663,7 +1662,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, @Override public IgniteInternalFuture<?> reloadAllAsync() { ctx.denyOnFlags(F.asList(LOCAL, READ)); - return reloadAllAsync(keySet(), (IgnitePredicate<CacheEntry<K, V>>[]) null); + return reloadAllAsync(keySet(), (IgnitePredicate<CacheEntry<K, V>>[])null); } /** @@ -1684,9 +1683,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, String taskName, final IgniteBiInClosure<K, V> vis) { return ctx.closures().callLocalSafe(new GPC<Object>() { - @Nullable - @Override - public Object call() { + @Nullable @Override public Object call() { try { ctx.store().loadAllFromStore(tx, keys, vis); } @@ -1897,7 +1894,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, /** {@inheritDoc} */ @Override public void evictAll(Collection<? extends K> keys) { - evictAll(keys, (IgnitePredicate<CacheEntry<K, V>>[]) null); + evictAll(keys, (IgnitePredicate<CacheEntry<K, V>>[])null); } /** {@inheritDoc} */ @@ -1931,9 +1928,8 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, if (ctx.config().getInterceptor() != null) fut = fut.chain(new CX1<IgniteInternalFuture<V>, V>() { - @Override - public V applyx(IgniteInternalFuture<V> f) throws IgniteCheckedException { - return (V) ctx.config().getInterceptor().onGet(key, f.get()); + @Override public V applyx(IgniteInternalFuture<V> f) throws IgniteCheckedException { + return (V)ctx.config().getInterceptor().onGet(key, f.get()); } }); @@ -2505,13 +2501,11 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, ctx.denyOnLocalRead(); return asyncOp(new AsyncInOp(drMap.keySet()) { - @Override - public IgniteInternalFuture<?> inOp(IgniteTxLocalAdapter<K, V> tx) { + @Override public IgniteInternalFuture<?> inOp(IgniteTxLocalAdapter<K, V> tx) { return tx.putAllDrAsync(ctx, drMap); } - @Override - public String toString() { + @Override public String toString() { return "putAllDrAsync [drMap=" + drMap + ']'; } }); @@ -2533,7 +2527,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, @Nullable @Override public EntryProcessorResult<T> op(IgniteTxLocalAdapter<K, V> tx) throws IgniteCheckedException { Map<? extends K, EntryProcessor<K, V, Object>> invokeMap = - Collections.singletonMap(key, (EntryProcessor<K, V, Object>) entryProcessor); + Collections.singletonMap(key, (EntryProcessor<K, V, Object>)entryProcessor); IgniteInternalFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> fut = tx.invokeAsync(ctx, invokeMap, args); @@ -2565,13 +2559,10 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, ctx.denyOnLocalRead(); return syncOp(new SyncOp<Map<K, EntryProcessorResult<T>>>(keys.size() == 1) { - @Nullable - @Override - public Map<K, EntryProcessorResult<T>> op(IgniteTxLocalAdapter tx) + @Nullable @Override public Map<K, EntryProcessorResult<T>> op(IgniteTxLocalAdapter tx) throws IgniteCheckedException { Map<? extends K, EntryProcessor<K, V, Object>> invokeMap = F.viewAsMap(keys, new C1<K, EntryProcessor<K, V, Object>>() { - @Override - public EntryProcessor apply(K k) { + @Override public EntryProcessor apply(K k) { return entryProcessor; } }); @@ -2602,7 +2593,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, IgniteInternalFuture<?> fut = asyncOp(new AsyncInOp(key) { @Override public IgniteInternalFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> inOp(IgniteTxLocalAdapter<K, V> tx) { Map<? extends K, EntryProcessor<K, V, Object>> invokeMap = - Collections.singletonMap(key, (EntryProcessor<K, V, Object>) entryProcessor); + Collections.singletonMap(key, (EntryProcessor<K, V, Object>)entryProcessor); return tx.invokeAsync(ctx, invokeMap, args); } @@ -2701,8 +2692,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, (IgniteInternalFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>>)fut; return fut0.chain(new CX1<IgniteInternalFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>>, Map<K, EntryProcessorResult<T>>>() { - @Override - public Map<K, EntryProcessorResult<T>> applyx( + @Override public Map<K, EntryProcessorResult<T>> applyx( IgniteInternalFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> fut) throws IgniteCheckedException { GridCacheReturn<Map<K, EntryProcessorResult<T>>> ret = fut.get(); @@ -2764,14 +2754,12 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, ctx.denyOnLocalRead(); return asyncOp(new AsyncOp<Boolean>(key) { - @Override - public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter<K, V> tx) { + @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter<K, V> tx) { return tx.putAllAsync(ctx, F.t(key, val), false, entry, ttl, filter).chain( - (IgniteClosure<IgniteInternalFuture<GridCacheReturn<V>>, Boolean>) RET2FLAG); + (IgniteClosure<IgniteInternalFuture<GridCacheReturn<V>>, Boolean>)RET2FLAG); } - @Override - public String toString() { + @Override public String toString() { return "putxAsync [key=" + key + ", val=" + val + ", filter=" + Arrays.toString(filter) + ']'; } }); @@ -2815,14 +2803,12 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, ctx.denyOnLocalRead(); IgniteInternalFuture<V> fut = ctx.wrapClone(asyncOp(new AsyncOp<V>(key) { - @Override - public IgniteInternalFuture<V> op(IgniteTxLocalAdapter<K, V> tx) { + @Override public IgniteInternalFuture<V> op(IgniteTxLocalAdapter<K, V> tx) { return tx.putAllAsync(ctx, F.t(key, val), true, null, -1, ctx.noPeekArray()) - .chain((IgniteClosure<IgniteInternalFuture<GridCacheReturn<V>>, V>) RET2VAL); + .chain((IgniteClosure<IgniteInternalFuture<GridCacheReturn<V>>, V>)RET2VAL); } - @Override - public String toString() { + @Override public String toString() { return "putIfAbsentAsync [key=" + key + ", val=" + val + ']'; } })); @@ -2849,13 +2835,11 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, ctx.denyOnLocalRead(); Boolean stored = syncOp(new SyncOp<Boolean>(true) { - @Override - public Boolean op(IgniteTxLocalAdapter<K, V> tx) throws IgniteCheckedException { + @Override public Boolean op(IgniteTxLocalAdapter<K, V> tx) throws IgniteCheckedException { return tx.putAllAsync(ctx, F.t(key, val), false, null, -1, ctx.noPeekArray()).get().success(); } - @Override - public String toString() { + @Override public String toString() { return "putxIfAbsent [key=" + key + ", val=" + val + ']'; } }); @@ -2964,13 +2948,11 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, ctx.denyOnLocalRead(); return syncOp(new SyncOp<Boolean>(true) { - @Override - public Boolean op(IgniteTxLocalAdapter<K, V> tx) throws IgniteCheckedException { + @Override public Boolean op(IgniteTxLocalAdapter<K, V> tx) throws IgniteCheckedException { return tx.putAllAsync(ctx, F.t(key, val), false, null, -1, ctx.hasPeekArray()).get().success(); } - @Override - public String toString() { + @Override public String toString() { return "replacex [key=" + key + ", val=" + val + ']'; } }); @@ -2988,14 +2970,12 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, ctx.denyOnLocalRead(); return asyncOp(new AsyncOp<Boolean>(key) { - @Override - public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter<K, V> tx) { + @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter<K, V> tx) { return tx.putAllAsync(ctx, F.t(key, val), false, null, -1, ctx.hasPeekArray()).chain( - (IgniteClosure<IgniteInternalFuture<GridCacheReturn<V>>, Boolean>) RET2FLAG); + (IgniteClosure<IgniteInternalFuture<GridCacheReturn<V>>, Boolean>)RET2FLAG); } - @Override - public String toString() { + @Override public String toString() { return "replacexAsync [key=" + key + ", val=" + val + ']'; } }); @@ -3119,13 +3099,11 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, ctx.denyOnLocalRead(); return asyncOp(new AsyncInOp(m.keySet()) { - @Override - public IgniteInternalFuture<?> inOp(IgniteTxLocalAdapter<K, V> tx) { + @Override public IgniteInternalFuture<?> inOp(IgniteTxLocalAdapter<K, V> tx) { return tx.putAllAsync(ctx, m, false, null, -1, filter); } - @Override - public String toString() { + @Override public String toString() { return "putAllAsync [map=" + m + ", filter=" + Arrays.toString(filter) + ']'; } }); @@ -3152,18 +3130,16 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, validateCacheKey(key); V prevVal = ctx.cloneOnFlag(syncOp(new SyncOp<V>(true) { - @Override - public V op(IgniteTxLocalAdapter<K, V> tx) throws IgniteCheckedException { + @Override public V op(IgniteTxLocalAdapter<K, V> tx) throws IgniteCheckedException { V ret = tx.removeAllAsync(ctx, Collections.singletonList(key), entry, true, filter).get().value(); if (ctx.config().getInterceptor() != null) - return (V) ctx.config().getInterceptor().onBeforeRemove(key, ret).get2(); + return (V)ctx.config().getInterceptor().onBeforeRemove(key, ret).get2(); return ret; } - @Override - public String toString() { + @Override public String toString() { return "remove [key=" + key + ", filter=" + Arrays.toString(filter) + ']'; } })); @@ -3278,13 +3254,11 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, ctx.denyOnLocalRead(); IgniteInternalFuture<Object> fut = asyncOp(new AsyncInOp(keys) { - @Override - public IgniteInternalFuture<?> inOp(IgniteTxLocalAdapter<K, V> tx) { + @Override public IgniteInternalFuture<?> inOp(IgniteTxLocalAdapter<K, V> tx) { return tx.removeAllAsync(ctx, keys, null, false, filter); } - @Override - public String toString() { + @Override public String toString() { return "removeAllAsync [keys=" + keys + ", filter=" + Arrays.toString(filter) + ']'; } }); @@ -3434,13 +3408,11 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, ctx.dr().onReceiveCacheEntriesReceived(drMap.size()); return asyncOp(new AsyncInOp(drMap.keySet()) { - @Override - public IgniteInternalFuture<?> inOp(IgniteTxLocalAdapter<K, V> tx) { + @Override public IgniteInternalFuture<?> inOp(IgniteTxLocalAdapter<K, V> tx) { return tx.removeAllDrAsync(ctx, drMap); } - @Override - public String toString() { + @Override public String toString() { return "removeAllDrASync [drMap=" + drMap + ']'; } }); @@ -3456,8 +3428,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, ctx.denyOnLocalRead(); return syncOp(new SyncOp<GridCacheReturn<V>>(true) { - @Override - public GridCacheReturn<V> op(IgniteTxLocalAdapter<K, V> tx) throws IgniteCheckedException { + @Override public GridCacheReturn<V> op(IgniteTxLocalAdapter<K, V> tx) throws IgniteCheckedException { // Register before hiding in the filter. if (ctx.deploymentEnabled()) ctx.deploy().registerClass(oldVal); @@ -3465,8 +3436,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, return tx.putAllAsync(ctx, F.t(key, newVal), true, null, -1, ctx.equalsPeekArray(oldVal)).get(); } - @Override - public String toString() { + @Override public String toString() { return "replace [key=" + key + ", oldVal=" + oldVal + ", newVal=" + newVal + ']'; } }); @@ -3547,8 +3517,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, validateCacheValue(val); boolean removed = syncOp(new SyncOp<Boolean>(true) { - @Override - public Boolean op(IgniteTxLocalAdapter<K, V> tx) throws IgniteCheckedException { + @Override public Boolean op(IgniteTxLocalAdapter<K, V> tx) throws IgniteCheckedException { // Register before hiding in the filter. if (ctx.deploymentEnabled()) ctx.deploy().registerClass(val); @@ -3556,14 +3525,13 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, K key0 = key; if (ctx.portableEnabled()) - key0 = (K) ctx.marshalToPortable(key); + key0 = (K)ctx.marshalToPortable(key); return tx.removeAllAsync(ctx, Collections.singletonList(key0), null, false, ctx.vararg(F.<K, V>cacheContainsPeek(val))).get().success(); } - @Override - public String toString() { + @Override public String toString() { return "remove [key=" + key + ", val=" + val + ']'; } }); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d4ddf85c/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 6fea05d..01f3a80 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 @@ -1678,7 +1678,7 @@ public class GridCacheSwapManager<K, V> extends GridCacheManagerAdapter<K, V> { private static <K, V> Iterator<Cache.Entry<K, V>> cacheEntryIterator(Iterator<Map.Entry<K, V>> it) { return F.iterator(it, new C1<Map.Entry<K, V>, Cache.Entry<K, V>>() { @Override public Cache.Entry<K, V> apply(Map.Entry<K, V> e) { - // Create Cache.Entry over Map.Entry to do not deserialize key/values if not needed. + // Create Cache.Entry over Map.Entry to do not deserialize keys/values if not needed. return new CacheEntryImpl0<>(e); } }, true); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d4ddf85c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekAbstractTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekAbstractTest.java deleted file mode 100644 index 80fec09..0000000 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekAbstractTest.java +++ /dev/null @@ -1,333 +0,0 @@ -/* - * 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.cache.affinity.*; -import org.apache.ignite.cache.eviction.fifo.*; -import org.apache.ignite.configuration.*; -import org.apache.ignite.internal.*; -import org.apache.ignite.spi.*; -import org.apache.ignite.spi.swapspace.file.*; - -import java.util.*; - -import static org.apache.ignite.cache.CacheDistributionMode.*; -import static org.apache.ignite.cache.CacheMode.*; -import static org.apache.ignite.cache.CachePeekMode.*; - -/** - * - */ -public abstract class IgniteCachePeekAbstractTest extends IgniteCacheAbstractTest { - /** */ - private static final int HEAP_ENTRIES = 10; - - /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(gridName); - - cfg.setSwapSpaceSpi(new FileSwapSpaceSpi()); - - return cfg; - } - - /** {@inheritDoc} */ - @Override protected CacheDistributionMode distributionMode() { - return PARTITIONED_ONLY; - } - - /** {@inheritDoc} */ - @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception { - CacheConfiguration ccfg = super.cacheConfiguration(gridName); - - ccfg.setMemoryMode(CacheMemoryMode.ONHEAP_TIERED); - - ccfg.setOffHeapMaxMemory(512); - - ccfg.setBackups(1); - - if (gridName.equals(getTestGridName(0))) - ccfg.setDistributionMode(NEAR_PARTITIONED); - - ccfg.setEvictionPolicy(new CacheFifoEvictionPolicy(HEAP_ENTRIES)); - - return ccfg; - } - - /** {@inheritDoc} */ - @Override protected boolean swapEnabled() { - return true; - } - - /** - * @throws Exception If failed. - */ - public void testLocalPeek() throws Exception { - if (cacheMode() == LOCAL) { - checkAffinityLocalCache(); - - checkStorage(0); - } - else { - checkAffinityPeek(0); - - checkAffinityPeek(1); - - checkStorage(0); - - checkStorage(1); - } - } - - /** - * @throws Exception If failed. - */ - private void checkAffinityLocalCache() throws Exception { - IgniteCache<Integer, String> cache0 = jcache(0); - - final String val = "1"; - - for (int i = 0; i < HEAP_ENTRIES; i++) - cache0.put(i, val); - - try { - for (int i = 0; i < HEAP_ENTRIES; i++) { - assertEquals(val, cache0.localPeek(i)); - assertEquals(val, cache0.localPeek(i, ALL)); - assertEquals(val, cache0.localPeek(i, PRIMARY)); - assertEquals(val, cache0.localPeek(i, BACKUP)); - assertEquals(val, cache0.localPeek(i, NEAR)); - } - } - finally { - for (int i = 0; i < HEAP_ENTRIES; i++) - cache0.remove(i); - } - } - - /** - * @param nodeIdx Node index. - * @throws Exception If failed. - */ - private void checkAffinityPeek(int nodeIdx) throws Exception { - IgniteCache<Integer, String> cache0 = jcache(nodeIdx); - - final String val = "1"; - - Integer key = null; - - try { - if (cacheMode() == REPLICATED) { - key = backupKey(cache0); - - cache0.put(key, val); - - assertEquals(val, cache0.localPeek(key, ALL)); - assertEquals(val, cache0.localPeek(key, BACKUP)); - assertNull(cache0.localPeek(key, NEAR)); - assertNull(cache0.localPeek(key, PRIMARY)); - } - else { - key = nearKey(cache0); - - cache0.put(key, val); - - boolean hasNearCache = nodeIdx == 0 ; - - if (hasNearCache) { - assertEquals(val, cache0.localPeek(key, NEAR)); - assertEquals(val, cache0.localPeek(key, ALL)); - } - else { - assertNull(cache0.localPeek(key, NEAR)); - assertNull(cache0.localPeek(key, ALL)); - } - - assertNull(cache0.localPeek(key, PRIMARY)); - assertNull(cache0.localPeek(key, BACKUP)); - } - - CacheAffinity<Integer> aff = ignite(0).affinity(null); - - for (int i = 0; i < gridCount(); i++) { - if (i == nodeIdx) - continue; - - IgniteCache<Integer, String> cache = jcache(i); - - assertNull(cache.localPeek(key, NEAR)); - - if (aff.isPrimary(ignite(i).cluster().localNode(), key)) { - assertEquals(val, cache.localPeek(key, PRIMARY)); - assertEquals(val, cache.localPeek(key, ALL)); - assertNull(cache.localPeek(key, BACKUP)); - assertNull(cache.localPeek(key, NEAR)); - } - else if (aff.isBackup(ignite(i).cluster().localNode(), key)) { - assertEquals(val, cache.localPeek(key, BACKUP)); - assertEquals(val, cache.localPeek(key, ALL)); - assertNull(cache.localPeek(key, PRIMARY)); - assertNull(cache.localPeek(key, NEAR)); - } - else { - assertNull(cache.localPeek(key, ALL)); - assertNull(cache.localPeek(key, PRIMARY)); - assertNull(cache.localPeek(key, BACKUP)); - assertNull(cache.localPeek(key, NEAR)); - } - } - } - finally { - if (key != null) - cache0.remove(key); - } - } - - /** - * @param nodeIdx Node index. - * @throws Exception If failed. - */ - private void checkStorage(int nodeIdx) throws Exception { - IgniteCache<Integer, String> cache0 = jcache(nodeIdx); - - List<Integer> keys = primaryKeys(cache0, 100, 10_000); - - try { - final String val = "test_value"; - - for (Integer key : keys) - cache0.put(key, val); - - FileSwapSpaceSpi swap = (FileSwapSpaceSpi)ignite(nodeIdx).configuration().getSwapSpaceSpi(); - - Set<Integer> swapKeys = new HashSet<>(); - - final String spaceName = "gg-swap-cache-dflt"; - - IgniteSpiCloseableIterator<Integer> it = swap.keyIterator(spaceName, null); - - assertNotNull(it); - - while (it.hasNext()) - assertTrue(swapKeys.add(it.next())); - - assertFalse(swapKeys.isEmpty()); - - assertTrue(swapKeys.size() + HEAP_ENTRIES < 100); - - Set<Integer> offheapKeys = new HashSet<>(); - - GridCacheAdapter<Integer, String> internalCache = - ((IgniteKernal)ignite(nodeIdx)).context().cache().internalCache(); - - Iterator<Map.Entry<Integer, String>> offheapIt; - - if (internalCache.context().isNear()) - offheapIt = internalCache.context().near().dht().context().swap().lazyOffHeapIterator(); - else - offheapIt = internalCache.context().swap().lazyOffHeapIterator(); - - while (offheapIt.hasNext()) { - Map.Entry<Integer, String> e = offheapIt.next(); - - assertTrue(offheapKeys.add(e.getKey())); - - assertFalse(swapKeys.contains(e.getKey())); - } - - assertFalse(offheapKeys.isEmpty()); - - Set<Integer> heapKeys = new HashSet<>(keys); - - heapKeys.removeAll(offheapKeys); - heapKeys.removeAll(swapKeys); - - assertFalse(heapKeys.isEmpty()); - - log.info("Keys [swap=" + swapKeys.size() + - ", offheap=" + offheapKeys.size() + - ", heap=" + heapKeys.size() + ']'); - - assertEquals(100, swapKeys.size() + offheapKeys.size() + heapKeys.size()); - - for (Integer key : swapKeys) { - assertEquals(val, cache0.localPeek(key, SWAP)); - assertEquals(val, cache0.localPeek(key, PRIMARY, SWAP)); - assertEquals(val, cache0.localPeek(key, ONHEAP, SWAP)); - assertEquals(val, cache0.localPeek(key, ONHEAP, OFFHEAP, SWAP)); - assertEquals(val, cache0.localPeek(key, PRIMARY, ONHEAP, SWAP)); - assertEquals(val, cache0.localPeek(key, PRIMARY, ONHEAP, OFFHEAP, SWAP)); - - if (cacheMode() == LOCAL) { - assertEquals(val, cache0.localPeek(key, SWAP, BACKUP)); - assertEquals(val, cache0.localPeek(key, SWAP, NEAR)); - } - else { - assertNull(cache0.localPeek(key, SWAP, BACKUP)); - assertNull(cache0.localPeek(key, SWAP, NEAR)); - } - - assertNull(cache0.localPeek(key, ONHEAP)); - assertNull(cache0.localPeek(key, OFFHEAP)); - } - - for (Integer key : offheapKeys) { - assertEquals(val, cache0.localPeek(key, OFFHEAP)); - assertEquals(val, cache0.localPeek(key, ONHEAP, OFFHEAP)); - assertEquals(val, cache0.localPeek(key, ONHEAP, SWAP, OFFHEAP)); - assertEquals(val, cache0.localPeek(key, PRIMARY, OFFHEAP)); - - if (cacheMode() == LOCAL) { - assertEquals(val, cache0.localPeek(key, OFFHEAP, BACKUP)); - assertEquals(val, cache0.localPeek(key, OFFHEAP, NEAR)); - } - else { - assertNull(cache0.localPeek(key, OFFHEAP, BACKUP)); - assertNull(cache0.localPeek(key, OFFHEAP, NEAR)); - } - - assertNull(cache0.localPeek(key, ONHEAP)); - assertNull(cache0.localPeek(key, SWAP)); - } - - for (Integer key : heapKeys) { - assertEquals(val, cache0.localPeek(key, ONHEAP)); - assertEquals(val, cache0.localPeek(key, SWAP, ONHEAP)); - assertEquals(val, cache0.localPeek(key, SWAP, OFFHEAP, ONHEAP)); - assertEquals(val, cache0.localPeek(key, PRIMARY, ONHEAP)); - - if (cacheMode() == LOCAL) { - assertEquals(val, cache0.localPeek(key, ONHEAP, BACKUP)); - assertEquals(val, cache0.localPeek(key, ONHEAP, NEAR)); - } - else { - assertNull(cache0.localPeek(key, ONHEAP, BACKUP)); - assertNull(cache0.localPeek(key, ONHEAP, NEAR)); - } - - assertNull(cache0.localPeek(key, SWAP)); - assertNull(cache0.localPeek(key, OFFHEAP)); - } - } - finally { - cache0.removeAll(new HashSet<>(keys)); - } - } -}