diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java index 1554e07..dfa0217 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java @@ -843,7 +843,7 @@ public class CacheMetricsImpl implements CacheMetrics { offHeapEvicts.incrementAndGet(); if (delegate != null) - delegate.onOffHeapRemove(); + delegate.onOffHeapEvict(); } /** 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 9329e94..1fc94ec 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 @@ -4096,21 +4096,22 @@ public abstract class GridCacheAdapter implements IgniteInternalCache swap eviction. + * + * @param entry Serialized swap entry. + * @param evictVer Version when entry was selected for eviction. + * @param obsoleteVer Obsolete version. + * @throws IgniteCheckedException If failed. + * @throws GridCacheEntryRemovedException If entry was removed. + * @return {@code True} if entry was obsoleted and written to swap. + */ + public boolean offheapSwapEvict(byte[] entry, GridCacheVersion evictVer, GridCacheVersion obsoleteVer) + throws IgniteCheckedException, GridCacheEntryRemovedException; + + /** * @return Value. * @throws IgniteCheckedException If failed to read from swap storage. * @throws GridCacheEntryRemovedException If entry was removed. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java index f60c0eb..3e0e2f9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java @@ -958,7 +958,7 @@ public class GridCacheEvictionManager extends GridCacheManagerAdapter { List locked = new ArrayList<>(keys.size()); - Set notRemove = null; + Set notRmv = null; Collection swapped = new ArrayList<>(keys.size()); @@ -990,10 +990,10 @@ public class GridCacheEvictionManager extends GridCacheManagerAdapter { locked.add(entry); if (entry.obsolete()) { - if (notRemove == null) - notRemove = new HashSet<>(); + if (notRmv == null) + notRmv = new HashSet<>(); - notRemove.add(entry); + notRmv.add(entry); continue; } @@ -1004,11 +1004,19 @@ public class GridCacheEvictionManager extends GridCacheManagerAdapter { GridCacheBatchSwapEntry swapEntry = entry.evictInBatchInternal(obsoleteVer); if (swapEntry != null) { + assert entry.obsolete() : entry; + swapped.add(swapEntry); if (log.isDebugEnabled()) log.debug("Entry was evicted [entry=" + entry + ", localNode=" + cctx.nodeId() + ']'); } + else if (!entry.obsolete()) { + if (notRmv == null) + notRmv = new HashSet<>(); + + notRmv.add(entry); + } } // Batch write to swap. @@ -1025,7 +1033,7 @@ public class GridCacheEvictionManager extends GridCacheManagerAdapter { // Remove entries and fire events outside the locks. for (GridCacheEntryEx entry : locked) { - if (entry.obsolete() && (notRemove == null || !notRemove.contains(entry))) { + if (entry.obsolete() && (notRmv == null || !notRmv.contains(entry))) { entry.onMarkedObsolete(); cache.removeEntry(entry); 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 eb4d864..f2bb646 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 @@ -433,6 +433,41 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme } /** {@inheritDoc} */ + @Override public boolean offheapSwapEvict(byte[] entry, GridCacheVersion evictVer, GridCacheVersion obsoleteVer) + throws IgniteCheckedException, GridCacheEntryRemovedException { + assert cctx.swap().swapEnabled() && cctx.swap().offHeapEnabled() : this; + + boolean obsolete; + + synchronized (this) { + checkObsolete(); + + if (hasReaders() || !isStartVersion()) + return false; + + GridCacheMvcc mvcc = mvccExtras(); + + if (mvcc != null && !mvcc.isEmpty(obsoleteVer)) + return false; + + if (cctx.swap().offheapSwapEvict(key, entry, partition(), evictVer)) { + assert !hasValueUnlocked() : this; + + obsolete = markObsolete0(obsoleteVer, false); + + assert obsolete : this; + } + else + obsolete = false; + } + + if (obsolete) + onMarkedObsolete(); + + return obsolete; + } + + /** {@inheritDoc} */ @Override public CacheObject unswap() throws IgniteCheckedException, GridCacheEntryRemovedException { return unswap(true); } @@ -536,7 +571,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme log.debug("Value did not change, skip write swap entry: " + this); if (cctx.swap().offheapEvictionEnabled()) - cctx.swap().enableOffheapEviction(key()); + cctx.swap().enableOffheapEviction(key(), partition()); return; } @@ -2988,7 +3023,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme synchronized (this) { checkObsolete(); - if (isNew() || (!preload && deletedUnlocked())) { + if ((isNew() && !cctx.swap().containsKey(key, partition())) || (!preload && deletedUnlocked())) { long expTime = expireTime < 0 ? CU.toExpireTime(ttl) : expireTime; val = cctx.kernalContext().cacheObjects().prepareForCache(val, cctx); @@ -3643,6 +3678,9 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme try { if (F.isEmptyOrNulls(filter)) { synchronized (this) { + if (obsoleteVersionExtras() != null) + return true; + CacheObject prev = saveValueForIndexUnlocked(); if (!hasReaders() && markObsolete0(obsoleteVer, false)) { @@ -3684,6 +3722,9 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme return false; synchronized (this) { + if (obsoleteVersionExtras() != null) + return true; + if (!v.equals(ver)) // Version has changed since entry passed the filter. Do it again. continue; @@ -3768,6 +3809,13 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme try { if (!hasReaders() && markObsolete0(obsoleteVer, false)) { if (!isStartVersion() && hasValueUnlocked()) { + if (cctx.offheapTiered() && hasOffHeapPointer()) { + if (cctx.swap().offheapEvictionEnabled()) + cctx.swap().enableOffheapEviction(key(), partition()); + + return null; + } + IgniteUuid valClsLdrId = null; IgniteUuid keyClsLdrId = null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapEntryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapEntryImpl.java index 81490a7..b7c66d3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapEntryImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapEntryImpl.java @@ -126,9 +126,9 @@ public class GridCacheSwapEntryImpl implements GridCacheSwapEntry { * @return Version. */ public static GridCacheVersion version(byte[] bytes) { - int off = VERSION_OFFSET; // Skip ttl, expire time. + long off = BYTE_ARR_OFF + VERSION_OFFSET; // Skip ttl, expire time. - boolean verEx = bytes[off++] != 0; + boolean verEx = UNSAFE.getByte(bytes, off++) != 0; return U.readVersion(bytes, off, verEx); } @@ -157,26 +157,6 @@ public class GridCacheSwapEntryImpl implements GridCacheSwapEntry { return new IgniteBiTuple<>(valBytes, type); } - /** - * @param bytes Entry bytes. - * @return Value bytes offset. - */ - public static int valueOffset(byte[] bytes) { - assert bytes.length > 40 : bytes.length; - - int off = VERSION_OFFSET; // Skip ttl, expire time. - - boolean verEx = bytes[off++] != 0; - - off += verEx ? VERSION_EX_SIZE : VERSION_SIZE; - - off += 5; // Byte array flag + array size. - - assert bytes.length >= off; - - return off; - } - /** {@inheritDoc} */ @Override public byte[] valueBytes() { if (valBytes != null) { 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 7fd6013..9b6381e 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 @@ -34,6 +34,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.internal.managers.swapspace.GridSwapSpaceManager; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.cache.version.GridCacheVersionAware; @@ -54,6 +55,7 @@ import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiInClosure; +import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.spi.swapspace.SwapKey; @@ -101,8 +103,16 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter { private final ReferenceQueue> itQ = new ReferenceQueue<>(); /** Soft iterator set. */ - private final Collection> itSet = - new GridConcurrentHashSet<>(); + private final Collection> itSet = new GridConcurrentHashSet<>(); + + /** {@code True} if offheap to swap eviction is possible. */ + private boolean offheapToSwapEvicts; + + /** Values to be evicted from offheap to swap. */ + private ThreadLocal>> offheapEvicts = new ThreadLocal<>(); + + /** First offheap eviction warning flag. */ + private volatile boolean firstEvictWarn; /** * @param enabled Flag to indicate if swap is enabled. @@ -127,9 +137,58 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter { } /** + * + */ + public void unwindOffheapEvicts() { + if (!offheapToSwapEvicts) + return; + + Collection> evicts = offheapEvicts.get(); + + if (evicts != null) { + GridCacheVersion obsoleteVer = cctx.versions().next(); + + for (IgniteBiTuple t : evicts) { + try { + byte[] kb = t.get1(); + byte[] vb = t.get2(); + + GridCacheVersion evictVer = GridCacheSwapEntryImpl.version(vb); + + KeyCacheObject key = cctx.toCacheKeyObject(kb); + + while (true) { + GridCacheEntryEx entry = cctx.cache().entryEx(key); + + try { + if (entry.offheapSwapEvict(vb, evictVer, obsoleteVer)) + cctx.cache().removeEntry(entry); + + break; + } + catch (GridCacheEntryRemovedException ignore) { + // Retry. + } + } + } + catch (GridDhtInvalidPartitionException e) { + // Skip entry. + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to unmarshal off-heap entry", e); + } + } + + offheapEvicts.set(null); + } + } + + /** * Initializes off-heap space. */ private void initOffHeap() { + assert offheapEnabled; + // Register big data usage. long max = cctx.config().getOffHeapMaxMemory(); @@ -137,43 +196,69 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter { int parts = cctx.config().getAffinity().partitions(); - GridOffHeapEvictListener lsnr = !swapEnabled && !offheapEnabled ? null : new GridOffHeapEvictListener() { - private volatile boolean firstEvictWarn; + GridOffHeapEvictListener lsnr; - @Override public void onEvict(int part, int hash, byte[] kb, byte[] vb) { - try { - if (!firstEvictWarn) - warnFirstEvict(); + if (swapEnabled) { + offheapToSwapEvicts = true; - writeToSwap(part, cctx.toCacheKeyObject(kb), vb); + lsnr = new GridOffHeapEvictListener() { + @Override public void onEvict(int part, int hash, byte[] kb, byte[] vb) { + assert offheapToSwapEvicts; - if (cctx.config().isStatisticsEnabled()) - cctx.cache().metrics0().onOffHeapEvict(); - } - catch (IgniteCheckedException e) { - U.error(log, "Failed to unmarshal off-heap entry [part=" + part + ", hash=" + hash + ']', e); - } - } + onOffheapEvict(); - private void warnFirstEvict() { - synchronized (this) { - if (firstEvictWarn) - return; + Collection> evicts = offheapEvicts.get(); - firstEvictWarn = true; + if (evicts == null) + offheapEvicts.set(evicts = new ArrayList<>()); + + evicts.add(new IgniteBiTuple<>(kb, vb)); } - U.warn(log, "Off-heap evictions started. You may wish to increase 'offHeapMaxMemory' in " + - "cache configuration [cache=" + cctx.name() + - ", offHeapMaxMemory=" + cctx.config().getOffHeapMaxMemory() + ']', - "Off-heap evictions started: " + cctx.name()); - } - }; + @Override public boolean removeEvicted() { + return false; + } + }; + } + else { + lsnr = new GridOffHeapEvictListener() { + @Override public void onEvict(int part, int hash, byte[] kb, byte[] vb) { + onOffheapEvict(); + } + + @Override public boolean removeEvicted() { + return true; + } + }; + } offheap.create(spaceName, parts, init, max, lsnr); } /** + * Warns on first evict from off-heap. + */ + private void onOffheapEvict() { + if (cctx.config().isStatisticsEnabled()) + cctx.cache().metrics0().onOffHeapEvict(); + + if (firstEvictWarn) + return; + + synchronized (this) { + if (firstEvictWarn) + return; + + firstEvictWarn = true; + } + + U.warn(log, "Off-heap evictions started. You may wish to increase 'offHeapMaxMemory' in " + + "cache configuration [cache=" + cctx.name() + + ", offHeapMaxMemory=" + cctx.config().getOffHeapMaxMemory() + ']', + "Off-heap evictions started: " + cctx.name()); + } + + /** * @return {@code True} if swap store is enabled. */ public boolean swapEnabled() { @@ -440,17 +525,16 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter { /** * @param key Key to check. + * @param part Partition. * @return {@code True} if key is contained. * @throws IgniteCheckedException If failed. */ - public boolean containsKey(KeyCacheObject key) throws IgniteCheckedException { + public boolean containsKey(KeyCacheObject key, int part) throws IgniteCheckedException { if (!offheapEnabled && !swapEnabled) return false; checkIteratorQueue(); - int part = cctx.affinity().partition(key); - // First check off-heap store. if (offheapEnabled) { boolean contains = offheap.contains(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext())); @@ -480,6 +564,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter { /** * @param key Key to read. + * @param keyBytes Key bytes. * @param part Key partition. * @param entryLocked {@code True} if cache entry is locked. * @param readOffheap Read offheap flag. @@ -966,6 +1051,46 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter { } /** + * @param key Key to move from offheap to swap. + * @param entry Serialized swap entry. + * @param part Partition. + * @param ver Expected entry version. + * @return {@code True} if removed. + * @throws IgniteCheckedException If failed. + */ + boolean offheapSwapEvict(final KeyCacheObject key, byte[] entry, int part, final GridCacheVersion ver) + throws IgniteCheckedException { + assert offheapEnabled; + + checkIteratorQueue(); + + boolean rmv = offheap.removex(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()), + new IgniteBiPredicate() { + @Override public boolean apply(Long ptr, Integer len) { + GridCacheVersion ver0 = GridCacheOffheapSwapEntry.version(ptr); + + return ver.equals(ver0); + } + } + ); + + if (rmv) { + Collection lsnrs = offheapLsnrs.get(part); + + if (lsnrs != null) { + GridCacheSwapEntry e = swapEntry(GridCacheSwapEntryImpl.unmarshal(entry)); + + for (GridCacheSwapListener lsnr : lsnrs) + lsnr.onEntryUnswapped(part, key, e); + } + + cctx.swap().writeToSwap(part, key, entry); + } + + return rmv; + } + + /** * @return {@code True} if offheap eviction is enabled. */ boolean offheapEvictionEnabled() { @@ -976,16 +1101,15 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter { * Enables eviction for offheap entry after {@link #readOffheapPointer} was called. * * @param key Key. + * @param part Partition. * @throws IgniteCheckedException If failed. */ - void enableOffheapEviction(final KeyCacheObject key) throws IgniteCheckedException { + void enableOffheapEviction(final KeyCacheObject key, int part) throws IgniteCheckedException { if (!offheapEnabled) return; checkIteratorQueue(); - int part = cctx.affinity().partition(key); - offheap.enableEviction(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext())); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java index 980971c..2d5698a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java @@ -1029,8 +1029,15 @@ public class GridCacheUtils { ctx.evicts().unwind(); - if (ctx.isNear()) - ctx.near().dht().context().evicts().unwind(); + ctx.swap().unwindOffheapEvicts(); + + if (ctx.isNear()) { + GridCacheContext dhtCtx = ctx.near().dht().context(); + + dhtCtx.evicts().unwind(); + + dhtCtx.swap().unwindOffheapEvicts(); + } ctx.ttl().expire(); } @@ -1041,14 +1048,8 @@ public class GridCacheUtils { public static void unwindEvicts(GridCacheSharedContext ctx) { assert ctx != null; - for (GridCacheContext cacheCtx : ctx.cacheContexts()) { - cacheCtx.evicts().unwind(); - - if (cacheCtx.isNear()) - cacheCtx.near().dht().context().evicts().unwind(); - - cacheCtx.ttl().expire(); - } + for (GridCacheContext cacheCtx : ctx.cacheContexts()) + unwindEvicts(cacheCtx); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java index c5f15cd..956f2bf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java @@ -262,8 +262,11 @@ public class GridDhtLocalPartition implements Comparable, map.put(entry.key(), entry); - if (!entry.isInternal()) + if (!entry.isInternal()) { + assert !entry.deleted() : entry; + mapPubSize.increment(); + } } /** @@ -271,7 +274,7 @@ public class GridDhtLocalPartition implements Comparable, */ @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter") void onRemoved(GridDhtCacheEntry entry) { - assert entry.obsolete(); + assert entry.obsolete() : entry; // Make sure to remove exactly this entry. synchronized (entry) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java index 5e183e9..fcb012f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java @@ -546,10 +546,13 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology { * @param updateSeq Update sequence. * @return Local partition. */ - private GridDhtLocalPartition localPartition(int p, AffinityTopologyVersion topVer, boolean create, boolean updateSeq) { - while (true) { - boolean belongs = cctx.affinity().localNode(p, topVer); + private GridDhtLocalPartition localPartition(int p, + AffinityTopologyVersion topVer, + boolean create, + boolean updateSeq) { + boolean belongs = create && cctx.affinity().localNode(p, topVer); + while (true) { GridDhtLocalPartition loc = locParts.get(p); if (loc != null && loc.state() == EVICTED) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java index 3f9decf..2048fdf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java @@ -234,6 +234,8 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd /** * Completeness callback. + * + * @return {@code True} if future was finished by this call. */ private boolean onComplete() { Throwable err0 = err.get(); @@ -457,6 +459,7 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd /** * @param reads Read entries. * @param writes Write entries. + * @throws IgniteCheckedException If failed. */ private void prepare( Iterable reads, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java index 21aaef2..ab6dc3c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java @@ -404,8 +404,13 @@ public final class GridNearTxFinishFuture extends GridCompoundIdentityFutu if (backup == null) { readyNearMappingFromBackup(mapping); + ClusterTopologyCheckedException cause = + new ClusterTopologyCheckedException("Backup node left grid: " + backupId); + + cause.retryReadyFuture(cctx.nextAffinityReadyFuture(tx.topologyVersion())); + mini.onDone(new IgniteTxRollbackCheckedException("Failed to commit transaction " + - "(backup has left grid): " + tx.xidVersion())); + "(backup has left grid): " + tx.xidVersion(), cause)); } else if (backup.isLocal()) { boolean committed = cctx.tm().txHandler().checkDhtRemoteTxCommitted(tx.xidVersion()); @@ -414,9 +419,15 @@ public final class GridNearTxFinishFuture extends GridCompoundIdentityFutu if (committed) mini.onDone(tx); - else + else { + ClusterTopologyCheckedException cause = + new ClusterTopologyCheckedException("Primary node left grid: " + nodeId); + + cause.retryReadyFuture(cctx.nextAffinityReadyFuture(tx.topologyVersion())); + mini.onDone(new IgniteTxRollbackCheckedException("Failed to commit transaction " + - "(transaction has been rolled back on backup node): " + tx.xidVersion())); + "(transaction has been rolled back on backup node): " + tx.xidVersion(), cause)); + } } else { GridDhtTxFinishRequest finishReq = new GridDhtTxFinishRequest( @@ -731,8 +742,19 @@ public final class GridNearTxFinishFuture extends GridCompoundIdentityFutu readyNearMappingFromBackup(m); - if (res.checkCommittedError() != null) - onDone(res.checkCommittedError()); + Throwable err = res.checkCommittedError(); + + if (err != null) { + if (err instanceof IgniteCheckedException) { + ClusterTopologyCheckedException cause = + ((IgniteCheckedException)err).getCause(ClusterTopologyCheckedException.class); + + if (cause != null) + cause.retryReadyFuture(cctx.nextAffinityReadyFuture(tx.topologyVersion())); + } + + onDone(err); + } else onDone(tx); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java index 9efa43a..756672a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java @@ -1049,6 +1049,7 @@ public class IgniteTxHandler { * * @param nodeId Node id that originated finish request. * @param req Request. + * @param {@code True} if transaction committed on this node. */ protected void sendReply(UUID nodeId, GridDhtTxFinishRequest req, boolean committed) { if (req.replyRequired()) { @@ -1057,9 +1058,13 @@ public class IgniteTxHandler { if (req.checkCommitted()) { res.checkCommitted(true); - if (!committed) + if (!committed) { + ClusterTopologyCheckedException cause = + new ClusterTopologyCheckedException("Primary node left grid."); + res.checkCommittedError(new IgniteTxRollbackCheckedException("Failed to commit transaction " + - "(transaction has been rolled back on backup node): " + req.version())); + "(transaction has been rolled back on backup node): " + req.version(), cause)); + } } try { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java index 024ea7c..492fa07 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java @@ -30,6 +30,7 @@ import org.apache.ignite.internal.util.offheap.GridOffHeapPartitionedMap; import org.apache.ignite.internal.util.typedef.CX2; import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.marshaller.Marshaller; import org.jetbrains.annotations.Nullable; @@ -261,13 +262,35 @@ public class GridOffHeapProcessor extends GridProcessorAdapter { * @return {@code true} If succeeded. * @throws IgniteCheckedException If failed. */ - public boolean removex(@Nullable String spaceName, int part, KeyCacheObject key, byte[] keyBytes) throws IgniteCheckedException { + public boolean removex(@Nullable String spaceName, int part, KeyCacheObject key, byte[] keyBytes) + throws IgniteCheckedException { GridOffHeapPartitionedMap m = offheap(spaceName); return m != null && m.removex(part, U.hash(key), keyBytes(key, keyBytes)); } /** + * Removes value from offheap space for the given key. + * + * @param spaceName Space name. + * @param part Partition. + * @param key Key. + * @param keyBytes Key bytes. + * @param p Value predicate (arguments are value address and value length). + * @return {@code true} If succeeded. + * @throws IgniteCheckedException If failed. + */ + public boolean removex(@Nullable String spaceName, + int part, + KeyCacheObject key, + byte[] keyBytes, + IgniteBiPredicate p) throws IgniteCheckedException { + GridOffHeapPartitionedMap m = offheap(spaceName); + + return m != null && m.removex(part, U.hash(key), keyBytes(key, keyBytes), p); + } + + /** * Gets iterator over contents of the given space. * * @param spaceName Space name. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapEvictListener.java b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapEvictListener.java index 4597be8..beafea4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapEvictListener.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapEvictListener.java @@ -30,4 +30,9 @@ public interface GridOffHeapEvictListener { * @param valBytes Value bytes. */ public void onEvict(int part, int hash, byte[] keyBytes, byte[] valBytes); + + /** + * @return {@code True} if entry selected for eviction should be immediately removed. + */ + public boolean removeEvicted(); } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMap.java b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMap.java index 1fcddd7..d14a582 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMap.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMap.java @@ -20,13 +20,14 @@ package org.apache.ignite.internal.util.offheap; import org.apache.ignite.internal.util.lang.GridCloseableIterator; import org.apache.ignite.internal.util.typedef.CX2; import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgniteBiTuple; import org.jetbrains.annotations.Nullable; /** * Off-heap map. */ -public interface GridOffHeapMap { +public interface GridOffHeapMap { /** * Gets partition this map belongs to. * @@ -102,6 +103,16 @@ public interface GridOffHeapMap { public boolean removex(int hash, byte[] keyBytes); /** + * Removes value from off-heap map without returning it. + * + * @param hash Hash. + * @param keyBytes Key bytes. + * @param p Value predicate (arguments are value address and value length). + * @return {@code True} if value was removed. + */ + public boolean removex(int hash, byte[] keyBytes, IgniteBiPredicate p); + + /** * Puts key and value bytes into the map potentially replacing * existing entry. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapFactory.java index 1a3d219..4dd911f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapFactory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapFactory.java @@ -32,8 +32,8 @@ public class GridOffHeapMapFactory { * @param initCap Initial capacity. * @return Off-heap map. */ - public static GridOffHeapMap unsafeMap(long initCap) { - return new GridUnsafeMap<>(128, 0.75f, initCap, 0, (short)0, null); + public static GridOffHeapMap unsafeMap(long initCap) { + return new GridUnsafeMap(128, 0.75f, initCap, 0, (short)0, null); } /** @@ -43,8 +43,8 @@ public class GridOffHeapMapFactory { * @param initCap Initial capacity. * @return Off-heap map. */ - public static GridOffHeapMap unsafeMap(int concurrency, long initCap) { - return new GridUnsafeMap<>(concurrency, 0.75f, initCap, 0, (short)0, null); + public static GridOffHeapMap unsafeMap(int concurrency, long initCap) { + return new GridUnsafeMap(concurrency, 0.75f, initCap, 0, (short)0, null); } /** @@ -55,8 +55,8 @@ public class GridOffHeapMapFactory { * @param initCap Initial capacity. * @return Off-heap map. */ - public static GridOffHeapMap unsafeMap(int concurrency, float load, long initCap) { - return new GridUnsafeMap<>(concurrency, load, initCap, 0, (short)0, null); + public static GridOffHeapMap unsafeMap(int concurrency, float load, long initCap) { + return new GridUnsafeMap(concurrency, load, initCap, 0, (short)0, null); } /** @@ -68,8 +68,8 @@ public class GridOffHeapMapFactory { * @param lruStripes Number of LRU stripes. * @return Off-heap map. */ - public static GridOffHeapMap unsafeMap(long initCap, long totalMem, short lruStripes) { - return new GridUnsafeMap<>(128, 0.75f, initCap, totalMem, lruStripes, null); + public static GridOffHeapMap unsafeMap(long initCap, long totalMem, short lruStripes) { + return new GridUnsafeMap(128, 0.75f, initCap, totalMem, lruStripes, null); } /** @@ -82,9 +82,9 @@ public class GridOffHeapMapFactory { * @param lsnr Optional eviction listener which gets notified every time an entry is evicted. * @return Off-heap map. */ - public static GridOffHeapMap unsafeMap(long initCap, long totalMem, short lruStripes, + public static GridOffHeapMap unsafeMap(long initCap, long totalMem, short lruStripes, @Nullable GridOffHeapEvictListener lsnr) { - return new GridUnsafeMap<>(128, 0.75f, initCap, totalMem, lruStripes, lsnr); + return new GridUnsafeMap(128, 0.75f, initCap, totalMem, lruStripes, lsnr); } /** @@ -98,9 +98,9 @@ public class GridOffHeapMapFactory { * @param lsnr Optional eviction listener which gets notified every time an entry is evicted. * @return Off-heap map. */ - public static GridOffHeapMap unsafeMap(int concurrency, long initCap, long totalMem, short lruStripes, + public static GridOffHeapMap unsafeMap(int concurrency, long initCap, long totalMem, short lruStripes, @Nullable GridOffHeapEvictListener lsnr) { - return new GridUnsafeMap<>(concurrency, 0.75f, initCap, totalMem, lruStripes, lsnr); + return new GridUnsafeMap(concurrency, 0.75f, initCap, totalMem, lruStripes, lsnr); } /** @@ -115,9 +115,9 @@ public class GridOffHeapMapFactory { * @param lsnr Optional eviction listener which gets notified every time an entry is evicted. * @return Off-heap map. */ - public static GridOffHeapMap unsafeMap(int concurrency, float load, long initCap, long totalMem, + public static GridOffHeapMap unsafeMap(int concurrency, float load, long initCap, long totalMem, short lruStripes, @Nullable GridOffHeapEvictListener lsnr) { - return new GridUnsafeMap<>(concurrency, load, initCap, totalMem, lruStripes, lsnr); + return new GridUnsafeMap(concurrency, load, initCap, totalMem, lruStripes, lsnr); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMap.java b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMap.java index 3afdfa9..c1e1bfa 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMap.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMap.java @@ -21,6 +21,7 @@ import java.util.Set; import org.apache.ignite.internal.util.lang.GridCloseableIterator; import org.apache.ignite.internal.util.typedef.CX2; import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgniteBiTuple; import org.jetbrains.annotations.Nullable; @@ -111,6 +112,17 @@ public interface GridOffHeapPartitionedMap { public boolean removex(int p, int hash, byte[] keyBytes); /** + * Removes value from off-heap map without returning it. + * + * @param part Partition. + * @param hash Hash. + * @param keyBytes Key bytes. + * @param p Value predicate (arguments are value address and value length). + * @return {@code True} if value was removed. + */ + public boolean removex(int part, int hash, byte[] keyBytes, IgniteBiPredicate p); + + /** * Puts key and value bytes into the map potentially replacing * existing entry. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMap.java b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMap.java index 40fb3e8..359d36c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMap.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMap.java @@ -33,6 +33,7 @@ import org.apache.ignite.internal.util.typedef.CX2; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgniteBiTuple; import org.jetbrains.annotations.Nullable; import org.jsr166.LongAdder8; @@ -42,7 +43,7 @@ import static org.apache.ignite.internal.util.offheap.GridOffHeapEvent.REHASH; /** * Off-heap map based on {@code Unsafe} implementation. */ -public class GridUnsafeMap implements GridOffHeapMap { +public class GridUnsafeMap implements GridOffHeapMap { /** Header size. */ private static final int HEADER = 4 /*hash*/ + 4 /*key-size*/ + 4 /*value-size*/ + 8 /*queue-address*/ + 8 /*next-address*/; @@ -77,7 +78,7 @@ public class GridUnsafeMap implements GridOffHeapMap { private final float load; /** Segments. */ - private final Segment[] segs; + private final Segment[] segs; /** Total memory. */ private final GridUnsafeMemory mem; @@ -111,6 +112,9 @@ public class GridUnsafeMap implements GridOffHeapMap { /** LRU poller. */ private final GridUnsafeLruPoller lruPoller; + /** */ + private final boolean rmvEvicted; + /** * @param concurrency Concurrency. * @param load Load factor. @@ -180,6 +184,8 @@ public class GridUnsafeMap implements GridOffHeapMap { } } }; + + rmvEvicted = evictLsnr == null || evictLsnr.removeEvicted(); } /** @@ -225,6 +231,8 @@ public class GridUnsafeMap implements GridOffHeapMap { segs = new Segment[size]; init(initCap, size); + + rmvEvicted = evictLsnr == null || evictLsnr.removeEvicted(); } /** @@ -247,7 +255,7 @@ public class GridUnsafeMap implements GridOffHeapMap { for (int i = 0; i < size; i++) { try { - segs[i] = new Segment<>(i, cap); + segs[i] = new Segment(i, cap); } catch (GridOffHeapOutOfMemoryException e) { destruct(); @@ -327,6 +335,11 @@ public class GridUnsafeMap implements GridOffHeapMap { } /** {@inheritDoc} */ + @Override public boolean removex(int hash, byte[] keyBytes, IgniteBiPredicate p) { + return segmentFor(hash).removex(hash, keyBytes, p); + } + + /** {@inheritDoc} */ @Override public boolean put(int hash, byte[] keyBytes, byte[] valBytes) { return segmentFor(hash).put(hash, keyBytes, valBytes); } @@ -559,7 +572,7 @@ public class GridUnsafeMap implements GridOffHeapMap { /** * Segment. */ - private class Segment { + private class Segment { /** Lock. */ private final ReadWriteLock lock = new ReentrantReadWriteLock(); @@ -1009,41 +1022,51 @@ public class GridUnsafeMap implements GridOffHeapMap { } if (cur != 0) { - long next = Entry.nextAddress(cur, mem); + long qAddr0 = Entry.queueAddress(cur, mem); - if (prev != 0) - Entry.nextAddress(prev, next, mem); // Relink. - else { - if (next == 0) - Bin.clear(binAddr, mem); - else - Bin.first(binAddr, next, mem); - } + assert qAddr == qAddr0 : "Queue node address mismatch " + + "[qAddr=" + qAddr + ", entryQueueAddr=" + qAddr + ']'; if (evictLsnr != null) { keyBytes = Entry.readKeyBytes(cur, mem); - // TODO: GG-8123: Inlined as a workaround. Revert when 7u60 is released. -// valBytes = Entry.readValueBytes(cur, mem); - { - int keyLen = Entry.readKeyLength(cur, mem); - int valLen = Entry.readValueLength(cur, mem); + int keyLen = Entry.readKeyLength(cur, mem); + int valLen = Entry.readValueLength(cur, mem); - valBytes = mem.readBytes(cur + HEADER + keyLen, valLen); - } + valBytes = mem.readBytes(cur + HEADER + keyLen, valLen); } - long a; + if (rmvEvicted) { + long a; - assert qAddr == (a = Entry.queueAddress(cur, mem)) : "Queue node address mismatch " + - "[qAddr=" + qAddr + ", entryQueueAddr=" + a + ']'; + assert qAddr == (a = Entry.queueAddress(cur, mem)) : "Queue node address mismatch " + + "[qAddr=" + qAddr + ", entryQueueAddr=" + a + ']'; - relSize = Entry.size(cur, mem); - relAddr = cur; + long next = Entry.nextAddress(cur, mem); - cnt--; + if (prev != 0) + Entry.nextAddress(prev, next, mem); // Relink. + else { + if (next == 0) + Bin.clear(binAddr, mem); + else + Bin.first(binAddr, next, mem); + } - totalCnt.decrement(); + relSize = Entry.size(cur, mem); + relAddr = cur; + + cnt--; + + totalCnt.decrement(); + } + else { + boolean clear = Entry.clearQueueAddress(cur, qAddr, mem); + + assert clear; + + relSize = Entry.size(cur, mem); + } } } } @@ -1251,7 +1274,7 @@ public class GridUnsafeMap implements GridOffHeapMap { */ @SuppressWarnings("TooBroadScope") byte[] remove(int hash, byte[] keyBytes) { - return remove(hash, keyBytes, true); + return remove(hash, keyBytes, true, null); } /** @@ -1260,17 +1283,28 @@ public class GridUnsafeMap implements GridOffHeapMap { * @return {@code True} if value was removed. */ boolean removex(int hash, byte[] keyBytes) { - return remove(hash, keyBytes, false) == EMPTY_BYTES; + return remove(hash, keyBytes, false, null) == EMPTY_BYTES; + } + + /** + * @param hash Hash. + * @param keyBytes Key bytes. + * @param p Value predicate. + * @return {@code True} if value was removed. + */ + boolean removex(int hash, byte[] keyBytes, IgniteBiPredicate p) { + return remove(hash, keyBytes, false, p) == EMPTY_BYTES; } /** * @param hash Hash. * @param keyBytes Key bytes. * @param retval {@code True} if need removed value. + * @param p Value predicate. * @return Removed value bytes. */ @SuppressWarnings("TooBroadScope") - byte[] remove(int hash, byte[] keyBytes, boolean retval) { + byte[] remove(int hash, byte[] keyBytes, boolean retval, @Nullable IgniteBiPredicate p) { int relSize = 0; long relAddr = 0; long qAddr = 0; @@ -1291,6 +1325,19 @@ public class GridUnsafeMap implements GridOffHeapMap { // If found match. if (Entry.keyEquals(cur, keyBytes, mem)) { + int keyLen = 0; + int valLen = 0; + + if (p != null) { + keyLen = Entry.readKeyLength(cur, mem); + valLen = Entry.readValueLength(cur, mem); + + long valPtr = cur + HEADER + keyLen; + + if (!p.apply(valPtr, valLen)) + return null; + } + if (prev != 0) Entry.nextAddress(prev, next, mem); // Relink. else { @@ -1300,18 +1347,16 @@ public class GridUnsafeMap implements GridOffHeapMap { Bin.first(binAddr, next, mem); } - // TODO: GG-8123: Inlined as a workaround. Revert when 7u60 is released. -// valBytes = retval ? Entry.readValueBytes(cur, mem) : EMPTY_BYTES; - { - if (retval) { - int keyLen = Entry.readKeyLength(cur, mem); - int valLen = Entry.readValueLength(cur, mem); - - valBytes = mem.readBytes(cur + HEADER + keyLen, valLen); + if (retval) { + if (keyLen == 0) { + keyLen = Entry.readKeyLength(cur, mem); + valLen = Entry.readValueLength(cur, mem); } - else - valBytes = EMPTY_BYTES; + + valBytes = mem.readBytes(cur + HEADER + keyLen, valLen); } + else + valBytes = EMPTY_BYTES; // Prepare release of memory. qAddr = Entry.queueAddress(cur, mem); @@ -1382,8 +1427,7 @@ public class GridUnsafeMap implements GridOffHeapMap { * @param keyBytes Key bytes. * @return Value pointer. */ - @Nullable - IgniteBiTuple valuePointer(int hash, byte[] keyBytes) { + @Nullable IgniteBiTuple valuePointer(int hash, byte[] keyBytes) { long binAddr = readLock(hash); try { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafePartitionedMap.java b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafePartitionedMap.java index 070da51..fb8ac14 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafePartitionedMap.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafePartitionedMap.java @@ -28,6 +28,7 @@ import org.apache.ignite.internal.util.offheap.GridOffHeapMap; import org.apache.ignite.internal.util.offheap.GridOffHeapPartitionedMap; import org.apache.ignite.internal.util.typedef.CX2; import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgniteBiTuple; import org.jetbrains.annotations.Nullable; import org.jsr166.LongAdder8; @@ -198,6 +199,14 @@ public class GridUnsafePartitionedMap implements GridOffHeapPartitionedMap { } /** {@inheritDoc} */ + @Override public boolean removex(int part, + int hash, + byte[] keyBytes, + IgniteBiPredicate p) { + return mapFor(part).removex(hash, keyBytes, p); + } + + /** {@inheritDoc} */ @Override public boolean put(int p, int hash, byte[] keyBytes, byte[] valBytes) { return mapFor(p).put(hash, keyBytes, valBytes); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTest.java index 271d8b1..214beb6 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTest.java @@ -26,6 +26,7 @@ import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMemoryMode; +import org.apache.ignite.cache.CachePeekMode; import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; @@ -53,6 +54,12 @@ public class CacheSwapUnswapGetTest extends GridCommonAbstractTest { /** */ private static final long DURATION = 30_000; + /** */ + private static final long OFFHEAP_MEM = 1000; + + /** */ + private static final int MAX_HEAP_SIZE = 100; + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); @@ -81,7 +88,7 @@ public class CacheSwapUnswapGetTest extends GridCommonAbstractTest { if (memMode == CacheMemoryMode.ONHEAP_TIERED) { LruEvictionPolicy plc = new LruEvictionPolicy(); - plc.setMaxSize(100); + plc.setMaxSize(MAX_HEAP_SIZE); ccfg.setEvictionPolicy(plc); } @@ -89,7 +96,7 @@ public class CacheSwapUnswapGetTest extends GridCommonAbstractTest { if (swap) { ccfg.setSwapEnabled(true); - ccfg.setOffHeapMaxMemory(1000); + ccfg.setOffHeapMaxMemory(OFFHEAP_MEM); } else ccfg.setOffHeapMaxMemory(0); @@ -133,6 +140,20 @@ public class CacheSwapUnswapGetTest extends GridCommonAbstractTest { /** * @throws Exception If failed. */ + public void testTxCacheOffheapSwapEvict() throws Exception { + swapUnswap(TRANSACTIONAL, CacheMemoryMode.ONHEAP_TIERED, true); + } + + /** + * @throws Exception If failed. + */ + public void testTxCacheOffheapTieredSwapEvict() throws Exception { + swapUnswap(TRANSACTIONAL, CacheMemoryMode.OFFHEAP_TIERED, true); + } + + /** + * @throws Exception If failed. + */ public void testAtomicCacheOffheapEvict() throws Exception { swapUnswap(ATOMIC, CacheMemoryMode.ONHEAP_TIERED, false); } @@ -145,6 +166,20 @@ public class CacheSwapUnswapGetTest extends GridCommonAbstractTest { } /** + * @throws Exception If failed. + */ + public void testAtomicCacheOffheapSwapEvict() throws Exception { + swapUnswap(ATOMIC, CacheMemoryMode.ONHEAP_TIERED, true); + } + + /** + * @throws Exception If failed. + */ + public void testAtomicCacheOffheapTieredSwapEvict() throws Exception { + swapUnswap(ATOMIC, CacheMemoryMode.OFFHEAP_TIERED, true); + } + + /** * @param atomicityMode Cache atomicity mode. * @param memMode Cache memory mode. * @param swap {@code True} if swap enabled. @@ -220,12 +255,56 @@ public class CacheSwapUnswapGetTest extends GridCommonAbstractTest { } }); - Thread.sleep(DURATION); + long endTime = System.currentTimeMillis() + DURATION; + + while (System.currentTimeMillis() < endTime) { + Thread.sleep(5000); + + log.info("Cache size [heap=" + cache.localSize(CachePeekMode.ONHEAP) + + ", offheap=" + cache.localSize(CachePeekMode.OFFHEAP) + + ", swap=" + cache.localSize(CachePeekMode.SWAP) + + ", total=" + cache.localSize() + + ", offheapMem=" + cache.metrics().getOffHeapAllocatedSize() + ']'); + } done.set(true); fut.get(); getFut.get(); + + for (Integer key : keys) { + String val = cache.get(key); + + assertNotNull(val); + } + + int onheapSize = cache.localSize(CachePeekMode.ONHEAP); + int offheapSize = cache.localSize(CachePeekMode.OFFHEAP); + int swapSize = cache.localSize(CachePeekMode.SWAP); + int total = cache.localSize(); + long offheapMem = cache.metrics().getOffHeapAllocatedSize(); + + log.info("Cache size [heap=" + onheapSize + + ", offheap=" + offheapSize + + ", swap=" + swapSize + + ", total=" + total + + ", offheapMem=" + offheapMem + ']'); + + assertTrue(total > 0); + + assertEquals(onheapSize + offheapSize + swapSize, total); + + if (memMode == CacheMemoryMode.OFFHEAP_TIERED) + assertEquals(0, onheapSize); + else + assertEquals(MAX_HEAP_SIZE, onheapSize); + + if (swap) { + assertTrue(swapSize > 0); + assertTrue(offheapMem <= OFFHEAP_MEM); + } + else + assertEquals(0, swapSize); } finally { done.set(true); 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 3e646d3..2a64963 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 @@ -2825,7 +2825,12 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract try { cache.clear(); - assertEquals(vals.get(first), cache.localPeek(first, ONHEAP)); + GridCacheContext cctx = context(0); + + GridCacheEntryEx entry = cctx.isNear() ? cctx.near().dht().peekEx(first) : + cctx.cache().peekEx(first); + + assertNotNull(entry); } finally { lock.unlock(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java index 31488e0..647746e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java @@ -21,38 +21,50 @@ import java.util.Collection; import java.util.HashSet; import java.util.Map; import java.util.concurrent.Callable; -import java.util.concurrent.CountDownLatch; +import java.util.concurrent.CyclicBarrier; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; +import java.util.concurrent.atomic.AtomicReference; import javax.cache.CacheException; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; -import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.CacheAtomicWriteOrderMode; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.util.lang.GridTuple; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi; import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.jsr166.ConcurrentHashMap8; import static org.apache.ignite.IgniteSystemProperties.IGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE; +import static org.apache.ignite.cache.CacheMode.PARTITIONED; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; /** * Tests that removes are not lost when topology changes. */ -public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstractSelfTest { +public abstract class GridCacheAbstractRemoveFailureTest extends GridCommonAbstractTest { + /** IP finder. */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + /** */ private static final int GRID_CNT = 3; /** Keys count. */ - private static final int KEYS_CNT = 10000; + private static final int KEYS_CNT = 10_000; /** Test duration. */ private static final long DUR = 90 * 1000L; @@ -66,36 +78,21 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra /** Start delay. */ private static final T2 START_DELAY = new T2<>(2000, 5000); - /** Node kill lock (used to prevent killing while cache data is compared). */ - private final Lock killLock = new ReentrantLock(); - /** */ - private CountDownLatch assertLatch; - - /** */ - private CountDownLatch updateLatch; - - /** Caches comparison request flag. */ - private volatile boolean cmp; - - /** */ - private String sizePropVal; + private static String sizePropVal; /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); - ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true); + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER).setForceServerMode(true); if (testClientNode() && getTestGridName(0).equals(gridName)) cfg.setClientMode(true); - return cfg; - } + cfg.setSwapSpaceSpi(new FileSwapSpaceSpi()); - /** {@inheritDoc} */ - @Override protected int gridCount() { - return GRID_CNT; + return cfg; } /** {@inheritDoc} */ @@ -104,6 +101,8 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra sizePropVal = System.getProperty(IGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE); System.setProperty(IGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE, "100000"); + + startGrids(GRID_CNT); } /** {@inheritDoc} */ @@ -111,15 +110,7 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra super.afterTestsStopped(); System.setProperty(IGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE, sizePropVal != null ? sizePropVal : ""); - } - /** {@inheritDoc} */ - @Override protected void beforeTest() throws Exception { - startGrids(gridCount()); - } - - /** {@inheritDoc} */ - @Override protected void afterTest() throws Exception { stopAllGrids(); } @@ -129,6 +120,28 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra } /** + * @return Cache mode. + */ + protected abstract CacheMode cacheMode(); + + /** + * @return Cache atomicity mode. + */ + protected abstract CacheAtomicityMode atomicityMode(); + + /** + * @return Near cache configuration. + */ + protected abstract NearCacheConfiguration nearCache(); + + /** + * @return Atomic cache write order mode. + */ + protected CacheAtomicWriteOrderMode atomicWriteOrderMode() { + return null; + } + + /** * @return {@code True} if test updates from client node. */ protected boolean testClientNode() { @@ -139,9 +152,49 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra * @throws Exception If failed. */ public void testPutAndRemove() throws Exception { - assertEquals(testClientNode(), (boolean)grid(0).configuration().isClientMode()); + putAndRemove(DUR, GridTestUtils.TestMemoryMode.HEAP); + } - final IgniteCache sndCache0 = grid(0).cache(null); + /** + * @throws Exception If failed. + */ + public void testPutAndRemoveOffheapEvict() throws Exception { + putAndRemove(30_000, GridTestUtils.TestMemoryMode.OFFHEAP_EVICT); + } + + /** + * @throws Exception If failed. + */ + public void testPutAndRemoveOffheapEvictSwap() throws Exception { + putAndRemove(30_000, GridTestUtils.TestMemoryMode.OFFHEAP_EVICT_SWAP); + } + + /** + * @param duration Test duration. + * @param memMode Memory mode. + * @throws Exception If failed. + */ + private void putAndRemove(long duration, GridTestUtils.TestMemoryMode memMode) throws Exception { + assertEquals(testClientNode(), (boolean) grid(0).configuration().isClientMode()); + + grid(0).destroyCache(null); + + CacheConfiguration ccfg = new CacheConfiguration<>(); + + ccfg.setWriteSynchronizationMode(FULL_SYNC); + + ccfg.setCacheMode(cacheMode()); + + if (cacheMode() == PARTITIONED) + ccfg.setBackups(1); + + ccfg.setAtomicityMode(atomicityMode()); + ccfg.setAtomicWriteOrderMode(atomicWriteOrderMode()); + ccfg.setNearConfiguration(nearCache()); + + GridTestUtils.setMemoryMode(null, ccfg, memMode, 100, 1024); + + final IgniteCache sndCache0 = grid(0).createCache(ccfg); final AtomicBoolean stop = new AtomicBoolean(); @@ -152,8 +205,12 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra // Expected values in cache. final Map> expVals = new ConcurrentHashMap8<>(); + final AtomicReference cmp = new AtomicReference<>(); + IgniteInternalFuture updateFut = GridTestUtils.runAsync(new Callable() { @Override public Void call() throws Exception { + Thread.currentThread().setName("update-thread"); + ThreadLocalRandom rnd = ThreadLocalRandom.current(); while (!stop.get()) { @@ -190,10 +247,14 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra cntr.addAndGet(100); - if (cmp) { - assertLatch.countDown(); + CyclicBarrier barrier = cmp.get(); + + if (barrier != null) { + log.info("Wait data check."); - updateLatch.await(); + barrier.await(60_000, TimeUnit.MILLISECONDS); + + log.info("Finished wait data check."); } } @@ -203,16 +264,21 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra IgniteInternalFuture killFut = GridTestUtils.runAsync(new Callable() { @Override public Void call() throws Exception { + Thread.currentThread().setName("restart-thread"); + while (!stop.get()) { U.sleep(random(KILL_DELAY.get1(), KILL_DELAY.get2())); - killLock.lock(); + killAndRestart(stop); - try { - killAndRestart(stop); - } - finally { - killLock.unlock(); + CyclicBarrier barrier = cmp.get(); + + if (barrier != null) { + log.info("Wait data check."); + + barrier.await(60_000, TimeUnit.MILLISECONDS); + + log.info("Finished wait data check."); } } @@ -221,7 +287,7 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra }); try { - long stopTime = DUR + U.currentTimeMillis() ; + long stopTime = duration + U.currentTimeMillis() ; long nextAssert = U.currentTimeMillis() + ASSERT_FREQ; @@ -241,31 +307,34 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra log.info("Operations/second: " + opsPerSecond); if (U.currentTimeMillis() >= nextAssert) { - updateLatch = new CountDownLatch(1); + CyclicBarrier barrier = new CyclicBarrier(3, new Runnable() { + @Override public void run() { + try { + cmp.set(null); - assertLatch = new CountDownLatch(1); + log.info("Checking cache content."); - cmp = true; + assertCacheContent(expVals); - killLock.lock(); + log.info("Finished check cache content."); + } + catch (Throwable e) { + log.error("Unexpected error: " + e, e); - try { - if (!assertLatch.await(60_000, TimeUnit.MILLISECONDS)) - throw new IgniteCheckedException("Failed to suspend thread executing updates."); + throw e; + } + } + }); - log.info("Checking cache content."); + log.info("Start cache content check."); - assertCacheContent(expVals); + cmp.set(barrier); - nextAssert = System.currentTimeMillis() + ASSERT_FREQ; - } - finally { - killLock.unlock(); + barrier.await(60_000, TimeUnit.MILLISECONDS); - updateLatch.countDown(); + log.info("Cache content check done."); - U.sleep(500); - } + nextAssert = System.currentTimeMillis() + ASSERT_FREQ; } } } @@ -278,18 +347,17 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra updateFut.get(); log.info("Test finished. Update errors: " + errCntr.get()); - } /** * @param stop Stop flag. * @throws Exception If failed. */ - void killAndRestart(AtomicBoolean stop) throws Exception { + private void killAndRestart(AtomicBoolean stop) throws Exception { if (stop.get()) return; - int idx = random(1, gridCount() + 1); + int idx = random(1, GRID_CNT + 1); log.info("Killing node " + idx); @@ -309,10 +377,9 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra /** * @param expVals Expected values in cache. - * @throws Exception If failed. */ @SuppressWarnings({"TooBroadScope", "ConstantIfStatement"}) - private void assertCacheContent(Map> expVals) throws Exception { + private void assertCacheContent(Map> expVals) { assert !expVals.isEmpty(); Collection failedKeys = new HashSet<>(); 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 c5c1c39..2f6ee8f 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 @@ -232,9 +232,15 @@ public class GridCacheMemoryModeSelfTest extends GridCommonAbstractTest { //putAll doTest(cache, offheapSwap, offheapEmpty, swapEmpty, new CIX1>() { @Override public void applyx(IgniteCache c) throws IgniteCheckedException { + putAll(c, 0, all / 2); + + putAll(c, all / 2 + 1, all - 1); + } + + private void putAll(IgniteCache c, int k1, int k2) { Map m = new HashMap<>(); - for (int i = 0; i < all; i++) + for (int i = k1; i <= k2; i++) m.put(valueOf(i), i); c.putAll(m); @@ -264,6 +270,7 @@ public class GridCacheMemoryModeSelfTest extends GridCommonAbstractTest { assertEquals(offheapSwap, c.localSize(CachePeekMode.OFFHEAP) + c.localSize(CachePeekMode.SWAP)); info("size: " + c.size()); + info("heap: " + c.localSize(CachePeekMode.ONHEAP)); info("offheap: " + c.localSize(CachePeekMode.OFFHEAP)); info("swap: " + c.localSize(CachePeekMode.SWAP)); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java index b401907..0c0d9c1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java @@ -60,6 +60,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.spi.failover.FailoverContext; import org.apache.ignite.spi.failover.always.AlwaysFailoverSpi; +import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; @@ -101,6 +102,9 @@ public class GridCachePutAllFailoverSelfTest extends GridCommonAbstractTest { /** Backups count. */ private int backups; + /** */ + private GridTestUtils.TestMemoryMode memMode = GridTestUtils.TestMemoryMode.HEAP; + /** Filter to include only worker nodes. */ private static final IgnitePredicate workerNodesFilter = new PN() { @SuppressWarnings("unchecked") @@ -202,6 +206,60 @@ public class GridCachePutAllFailoverSelfTest extends GridCommonAbstractTest { checkPutAllFailoverColocated(false, 5, 2); } + /** + * @throws Exception If failed. + */ + public void testPutAllFailoverColocatedNearEnabledTwoBackupsSwap() throws Exception { + memMode = GridTestUtils.TestMemoryMode.SWAP; + + checkPutAllFailoverColocated(true, 5, 2); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllFailoverColocatedTwoBackupsSwap() throws Exception { + memMode = GridTestUtils.TestMemoryMode.SWAP; + + checkPutAllFailoverColocated(false, 5, 2); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllFailoverColocatedNearEnabledTwoBackupsOffheapTiered() throws Exception { + memMode = GridTestUtils.TestMemoryMode.OFFHEAP_TIERED; + + checkPutAllFailoverColocated(true, 5, 2); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllFailoverColocatedNearEnabledTwoBackupsOffheapTieredSwap() throws Exception { + memMode = GridTestUtils.TestMemoryMode.OFFHEAP_TIERED_SWAP; + + checkPutAllFailoverColocated(true, 5, 2); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllFailoverColocatedNearEnabledTwoBackupsOffheapEvict() throws Exception { + memMode = GridTestUtils.TestMemoryMode.OFFHEAP_EVICT; + + checkPutAllFailoverColocated(true, 5, 2); + } + + /** + * @throws Exception If failed. + */ + public void testPutAllFailoverColocatedNearEnabledTwoBackupsOffheapEvictSwap() throws Exception { + memMode = GridTestUtils.TestMemoryMode.OFFHEAP_EVICT_SWAP; + + checkPutAllFailoverColocated(true, 5, 2); + } + /** {@inheritDoc} */ @Override protected long getTestTimeout() { return super.getTestTimeout() * 5; @@ -696,6 +754,8 @@ public class GridCachePutAllFailoverSelfTest extends GridCommonAbstractTest { cacheCfg.setWriteSynchronizationMode(FULL_SYNC); + GridTestUtils.setMemoryMode(cfg, cacheCfg, memMode, 1000, 10 * 1024); + cfg.setCacheConfiguration(cacheCfg); } else diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java index d9510e6..1fef4d5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java @@ -803,6 +803,12 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr } /** {@inheritDoc} */ + @Override public boolean offheapSwapEvict(byte[] vb, GridCacheVersion evictVer, GridCacheVersion obsoleteVer) + throws IgniteCheckedException, GridCacheEntryRemovedException { + return false; + } + + /** {@inheritDoc} */ @Override public CacheObject unswap(boolean needVal) throws IgniteCheckedException { return null; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java index 757dce8..6f0565b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java @@ -36,11 +36,13 @@ import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteTransactions; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheRebalanceMode; +import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.apache.ignite.transactions.Transaction; import org.apache.ignite.transactions.TransactionConcurrency; @@ -110,6 +112,9 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs private int retries = DFLT_RETRIES; /** */ + private GridTestUtils.TestMemoryMode memMode = GridTestUtils.TestMemoryMode.HEAP; + + /** */ private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); /** {@inheritDoc} */ @@ -129,11 +134,23 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs c.setDiscoverySpi(disco); + CacheConfiguration ccfg = cacheConfiguration(); + + GridTestUtils.setMemoryMode(c, ccfg, memMode, 100, 1024); + + c.setCacheConfiguration(ccfg); + return c; } + /** + * @return Cache configuration. + */ + protected abstract CacheConfiguration cacheConfiguration(); + /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { + // No-op. } /** {@inheritDoc} */ @@ -280,7 +297,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs long duration = 30000; - checkRestartWithTx(duration, 1, 1); + checkRestartWithTx(duration, 1, 1, 3); } /** @@ -310,7 +327,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs long duration = 30000; - checkRestartWithTx(duration, 1, 1); + checkRestartWithTx(duration, 1, 1, 3); } /** @@ -340,7 +357,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs long duration = 60000; - checkRestartWithTx(duration, 2, 2); + checkRestartWithTx(duration, 2, 2, 3); } /** @@ -361,6 +378,59 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs /** * @throws Exception If failed. */ + public void testRestartWithPutFourNodesOneBackupsSwap() throws Throwable { + restartWithPutFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.SWAP); + } + + /** + * @throws Exception If failed. + */ + public void testRestartWithPutFourNodesOneBackupsOffheapTiered() throws Throwable { + restartWithPutFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.OFFHEAP_TIERED); + } + + /** + * @throws Exception If failed. + */ + public void testRestartWithPutFourNodesOneBackupsOffheapTieredSwap() throws Throwable { + restartWithPutFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.OFFHEAP_TIERED_SWAP); + } + + /** + * @throws Exception If failed. + */ + public void testRestartWithPutFourNodesOneBackupsOffheapEvict() throws Throwable { + restartWithPutFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.OFFHEAP_EVICT); + } + + /** + * @throws Exception If failed. + */ + public void testRestartWithPutFourNodesOneBackupsOffheapEvictSwap() throws Throwable { + restartWithPutFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.OFFHEAP_EVICT_SWAP); + } + + /** + * @param memMode Memory mode. + * @throws Throwable If failed. + */ + private void restartWithPutFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode memMode) + throws Throwable { + backups = 1; + nodeCnt = 4; + keyCnt = 100_000; + partitions = 29; + rebalancMode = ASYNC; + this.memMode = memMode; + + long duration = 30_000; + + checkRestartWithPut(duration, 2, 2); + } + + /** + * @throws Exception If failed. + */ public void testRestartWithTxFourNodesOneBackups() throws Throwable { backups = 1; nodeCnt = 4; @@ -370,7 +440,59 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs long duration = 60000; - checkRestartWithTx(duration, 2, 2); + checkRestartWithTx(duration, 2, 2, 3); + } + + /** + * @throws Exception If failed. + */ + public void testRestartWithTxFourNodesOneBackupsSwap() throws Throwable { + restartWithTxFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.SWAP); + } + + /** + * @throws Exception If failed. + */ + public void testRestartWithTxFourNodesOneBackupsOffheapTiered() throws Throwable { + restartWithTxFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.OFFHEAP_TIERED); + } + + /** + * @throws Exception If failed. + */ + public void testRestartWithTxFourNodesOneBackupsOffheapTieredSwap() throws Throwable { + restartWithTxFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.OFFHEAP_TIERED_SWAP); + } + + /** + * @throws Exception If failed. + */ + public void testRestartWithTxFourNodesOneBackupsOffheapEvict() throws Throwable { + restartWithTxFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.OFFHEAP_EVICT); + } + + /** + * @throws Exception If failed. + */ + public void testRestartWithTxFourNodesOneBackupsOffheapEvictSwap() throws Throwable { + restartWithTxFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode.OFFHEAP_EVICT_SWAP); + } + + /** + * @param memMode Memory mode. + * @throws Throwable If failed. + */ + private void restartWithTxFourNodesOneBackupsWithMemoryMode(GridTestUtils.TestMemoryMode memMode) throws Throwable { + backups = 1; + nodeCnt = 4; + keyCnt = 100_000; + partitions = 29; + rebalancMode = ASYNC; + this.memMode = memMode; + + long duration = 30_000; + + checkRestartWithTx(duration, 2, 2, 100); } /** @@ -400,7 +522,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs long duration = 90000; - checkRestartWithTx(duration, 3, 3); + checkRestartWithTx(duration, 3, 3, 3); } /** @@ -430,7 +552,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs long duration = 90000; - checkRestartWithTx(duration, 4, 4); + checkRestartWithTx(duration, 4, 4, 3); } /** @@ -460,7 +582,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs long duration = 90000; - checkRestartWithTx(duration, 5, 5); + checkRestartWithTx(duration, 5, 5, 3); } /** @@ -499,7 +621,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs * @param restartThreads Restart threads count. * @throws Exception If failed. */ - public void checkRestartWithPut(long duration, int putThreads, int restartThreads) throws Throwable { + private void checkRestartWithPut(long duration, int putThreads, int restartThreads) throws Throwable { final long endTime = System.currentTimeMillis() + duration; final AtomicReference err = new AtomicReference<>(); @@ -613,9 +735,13 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs * @param duration Test duration. * @param putThreads Put threads count. * @param restartThreads Restart threads count. + * @param txKeys Keys per transaction. * @throws Exception If failed. */ - public void checkRestartWithTx(long duration, int putThreads, int restartThreads) throws Throwable { + private void checkRestartWithTx(long duration, + int putThreads, + int restartThreads, + final int txKeys) throws Throwable { if (atomicityMode() == ATOMIC) return; @@ -634,8 +760,6 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs final CyclicBarrier barrier = new CyclicBarrier(putThreads + restartThreads); - final int txKeys = 3; - for (int i = 0; i < putThreads; i++) { final int gridIdx = i; @@ -699,6 +823,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs if (c % logFreq == 0) { info(">>> Tx iteration finished [cnt=" + c + + ", cacheSize=" + cache.localSize() + ", keys=" + keys + ", locNodeId=" + locNodeId + ']'); } @@ -773,7 +898,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs * @param restartThreads Restart threads count. * @throws Exception If failed. */ - public void checkRestartWithTxPutAll(long duration, int putThreads, int restartThreads) throws Throwable { + private void checkRestartWithTxPutAll(long duration, int putThreads, int restartThreads) throws Throwable { if (atomicityMode() == ATOMIC) return; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtAtomicRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtAtomicRemoveFailureTest.java index b7f52ba..bee94a2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtAtomicRemoveFailureTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtAtomicRemoveFailureTest.java @@ -17,11 +17,13 @@ package org.apache.ignite.internal.processors.cache.distributed.dht; +import org.apache.ignite.cache.CacheAtomicWriteOrderMode; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; -import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.internal.processors.cache.GridCacheAbstractRemoveFailureTest; +import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.CLOCK; import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; import static org.apache.ignite.cache.CacheMode.PARTITIONED; @@ -40,12 +42,12 @@ public class GridCacheDhtAtomicRemoveFailureTest extends GridCacheAbstractRemove } /** {@inheritDoc} */ - @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception { - CacheConfiguration cfg = super.cacheConfiguration(gridName); - - cfg.setNearConfiguration(null); - cfg.setBackups(1); + @Override protected CacheAtomicWriteOrderMode atomicWriteOrderMode() { + return CLOCK; + } - return cfg; + /** {@inheritDoc} */ + @Override protected NearCacheConfiguration nearCache() { + return null; } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtRemoveFailureTest.java index 1c351ff..a13ba30 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtRemoveFailureTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtRemoveFailureTest.java @@ -17,10 +17,12 @@ package org.apache.ignite.internal.processors.cache.distributed.dht; +import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; -import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.internal.processors.cache.GridCacheAbstractRemoveFailureTest; +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; import static org.apache.ignite.cache.CacheMode.PARTITIONED; /** @@ -33,12 +35,12 @@ public class GridCacheDhtRemoveFailureTest extends GridCacheAbstractRemoveFailur } /** {@inheritDoc} */ - @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception { - CacheConfiguration cfg = super.cacheConfiguration(gridName); - - cfg.setNearConfiguration(null); - cfg.setBackups(1); + @Override protected CacheAtomicityMode atomicityMode() { + return TRANSACTIONAL; + } - return cfg; + /** {@inheritDoc} */ + @Override protected NearCacheConfiguration nearCache() { + return null; } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java index dc78003..4302486 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java @@ -199,7 +199,7 @@ public class GridCacheTxNodeFailureSelfTest extends GridCommonAbstractTest { try { final Ignite ignite = ignite(0); - final IgniteCache cache = ignite.cache(null); + final IgniteCache cache = ignite.cache(null).withNoRetries(); final int key = generateKey(ignite, backup); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java index 4c52e6f..7425e23 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java @@ -40,7 +40,7 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; -import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.apache.ignite.transactions.Transaction; import org.apache.ignite.transactions.TransactionConcurrency; @@ -51,6 +51,9 @@ import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; import static org.apache.ignite.cache.CacheMode.PARTITIONED; import static org.apache.ignite.cache.CacheMode.REPLICATED; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.testframework.GridTestUtils.TestMemoryMode; +import static org.apache.ignite.testframework.GridTestUtils.runMultiThreadedAsync; +import static org.apache.ignite.testframework.GridTestUtils.setMemoryMode; import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC; import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; @@ -86,6 +89,8 @@ public class IgniteCacheCrossCacheTxFailoverTest extends GridCommonAbstractTest if (gridName.equals(getTestGridName(GRID_CNT - 1))) cfg.setClientMode(true); + cfg.setSwapSpaceSpi(new FileSwapSpaceSpi()); + return cfg; } @@ -107,9 +112,13 @@ public class IgniteCacheCrossCacheTxFailoverTest extends GridCommonAbstractTest * @param name Cache name. * @param cacheMode Cache mode. * @param parts Number of partitions. + * @param memMode Memory mode. * @return Cache configuration. */ - private CacheConfiguration cacheConfiguration(String name, CacheMode cacheMode, int parts) { + private CacheConfiguration cacheConfiguration(String name, + CacheMode cacheMode, + int parts, + TestMemoryMode memMode) { CacheConfiguration ccfg = new CacheConfiguration(); ccfg.setName(name); @@ -122,6 +131,8 @@ public class IgniteCacheCrossCacheTxFailoverTest extends GridCommonAbstractTest ccfg.setAffinity(new RendezvousAffinityFunction(false, parts)); + setMemoryMode(null, ccfg, memMode, 100, 1024); + return ccfg; } @@ -134,49 +145,63 @@ public class IgniteCacheCrossCacheTxFailoverTest extends GridCommonAbstractTest * @throws Exception If failed. */ public void testCrossCachePessimisticTxFailover() throws Exception { - crossCacheTxFailover(PARTITIONED, true, PESSIMISTIC, REPEATABLE_READ); + crossCacheTxFailover(PARTITIONED, true, PESSIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP); + } + + /** + * @throws Exception If failed. + */ + public void testCrossCachePessimisticTxFailoverOffheapSwap() throws Exception { + crossCacheTxFailover(PARTITIONED, true, PESSIMISTIC, REPEATABLE_READ, TestMemoryMode.OFFHEAP_EVICT_SWAP); } /** * @throws Exception If failed. */ public void testCrossCachePessimisticTxFailoverDifferentAffinity() throws Exception { - crossCacheTxFailover(PARTITIONED, false, PESSIMISTIC, REPEATABLE_READ); + crossCacheTxFailover(PARTITIONED, false, PESSIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP); } /** * @throws Exception If failed. */ public void testCrossCacheOptimisticTxFailover() throws Exception { - crossCacheTxFailover(PARTITIONED, true, OPTIMISTIC, REPEATABLE_READ); + crossCacheTxFailover(PARTITIONED, true, OPTIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP); + } + + /** + * @throws Exception If failed. + */ + public void testCrossCacheOptimisticTxFailoverOffheapSwap() throws Exception { + crossCacheTxFailover(PARTITIONED, true, OPTIMISTIC, REPEATABLE_READ, TestMemoryMode.OFFHEAP_EVICT_SWAP); } /** * @throws Exception If failed. */ public void testCrossCacheOptimisticTxFailoverDifferentAffinity() throws Exception { - crossCacheTxFailover(PARTITIONED, false, OPTIMISTIC, REPEATABLE_READ); + crossCacheTxFailover(PARTITIONED, false, OPTIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP); } /** * @throws Exception If failed. */ public void testCrossCachePessimisticTxFailoverReplicated() throws Exception { - crossCacheTxFailover(REPLICATED, true, PESSIMISTIC, REPEATABLE_READ); + crossCacheTxFailover(REPLICATED, true, PESSIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP); } /** * @throws Exception If failed. */ public void testCrossCacheOptimisticTxFailoverReplicated() throws Exception { - crossCacheTxFailover(REPLICATED, true, OPTIMISTIC, REPEATABLE_READ); + crossCacheTxFailover(REPLICATED, true, OPTIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP); } /** * @throws Exception If failed. */ public void testCrossCachePessimisticTxFailoverDifferentAffinityReplicated() throws Exception { - crossCacheTxFailover(PARTITIONED, false, PESSIMISTIC, REPEATABLE_READ); + crossCacheTxFailover(PARTITIONED, false, PESSIMISTIC, REPEATABLE_READ, TestMemoryMode.HEAP); } /** @@ -184,23 +209,25 @@ public class IgniteCacheCrossCacheTxFailoverTest extends GridCommonAbstractTest * @param sameAff If {@code false} uses different number of partitions for caches. * @param concurrency Transaction concurrency. * @param isolation Transaction isolation. + * @param memMode Memory mode. * @throws Exception If failed. */ private void crossCacheTxFailover(CacheMode cacheMode, boolean sameAff, final TransactionConcurrency concurrency, - final TransactionIsolation isolation) throws Exception { + final TransactionIsolation isolation, + TestMemoryMode memMode) throws Exception { IgniteKernal ignite0 = (IgniteKernal)ignite(0); final AtomicBoolean stop = new AtomicBoolean(); try { - ignite0.createCache(cacheConfiguration(CACHE1, cacheMode, 256)); - ignite0.createCache(cacheConfiguration(CACHE2, cacheMode, sameAff ? 256 : 128)); + ignite0.createCache(cacheConfiguration(CACHE1, cacheMode, 256, memMode)); + ignite0.createCache(cacheConfiguration(CACHE2, cacheMode, sameAff ? 256 : 128, memMode)); final AtomicInteger threadIdx = new AtomicInteger(); - IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(new Callable() { + IgniteInternalFuture fut = runMultiThreadedAsync(new Callable() { @Override public Void call() throws Exception { int idx = threadIdx.getAndIncrement(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java index 8a72bb9..943caeb 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java @@ -24,35 +24,54 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.atomic.AtomicBoolean; +import javax.cache.Cache; +import javax.cache.configuration.Factory; +import javax.cache.integration.CacheLoaderException; +import javax.cache.integration.CacheWriterException; import javax.cache.processor.EntryProcessorResult; import javax.cache.processor.MutableEntry; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.CacheAtomicWriteOrderMode; +import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheEntryProcessor; import org.apache.ignite.cache.CachePartialUpdateException; +import org.apache.ignite.cache.store.CacheStore; +import org.apache.ignite.cache.store.CacheStoreAdapter; import org.apache.ignite.configuration.AtomicConfiguration; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; -import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi; import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.CLOCK; import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; import static org.apache.ignite.cache.CacheRebalanceMode.SYNC; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.testframework.GridTestUtils.TestMemoryMode; +import static org.apache.ignite.testframework.GridTestUtils.runAsync; /** * */ -public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstractSelfTest { - /** {@inheritDoc} */ - @Override protected int gridCount() { - return 4; - } +public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCommonAbstractTest { + /** IP finder. */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final long DURATION = 60_000; + + /** */ + protected static final int GRID_CNT = 4; /** * @return Keys count for the test. @@ -61,14 +80,29 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr return 10_000; } - /** {@inheritDoc} */ - @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception { - CacheConfiguration cfg = super.cacheConfiguration(gridName); + /** + * @param memMode Memory mode. + * @param store If {@code true} adds cache store. + * @return Cache configuration. + * @throws Exception If failed. + */ + @SuppressWarnings("unchecked") + protected CacheConfiguration cacheConfiguration(TestMemoryMode memMode, boolean store) throws Exception { + CacheConfiguration cfg = new CacheConfiguration(); + cfg.setAtomicityMode(atomicityMode()); + cfg.setWriteSynchronizationMode(FULL_SYNC); cfg.setAtomicWriteOrderMode(writeOrderMode()); cfg.setBackups(1); cfg.setRebalanceMode(SYNC); + if (store) { + cfg.setCacheStoreFactory(new TestStoreFactory()); + cfg.setWriteThrough(true); + } + + GridTestUtils.setMemoryMode(null, cfg, memMode, 100, 1024); + return cfg; } @@ -76,15 +110,47 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER); + + ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1); + AtomicConfiguration acfg = new AtomicConfiguration(); acfg.setBackups(1); cfg.setAtomicConfiguration(acfg); + cfg.setSwapSpaceSpi(new FileSwapSpaceSpi()); + return cfg; } + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGridsMultiThreaded(GRID_CNT); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + ignite(0).destroyCache(null); + } + + /** + * @return Cache atomicity mode. + */ + protected abstract CacheAtomicityMode atomicityMode(); + /** * @return Write order mode. */ @@ -96,47 +162,79 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr * @throws Exception If failed. */ public void testPut() throws Exception { - checkRetry(Test.PUT); + checkRetry(Test.PUT, TestMemoryMode.HEAP, false); + } + + /** + * @throws Exception If failed. + */ + public void testPutStoreEnabled() throws Exception { + checkRetry(Test.PUT, TestMemoryMode.HEAP, true); } /** * @throws Exception If failed. */ public void testPutAll() throws Exception { - checkRetry(Test.PUT_ALL); + checkRetry(Test.PUT_ALL, TestMemoryMode.HEAP, false); } /** * @throws Exception If failed. */ public void testPutAsync() throws Exception { - checkRetry(Test.PUT_ASYNC); + checkRetry(Test.PUT_ASYNC, TestMemoryMode.HEAP, false); + } + + /** + * @throws Exception If failed. + */ + public void testPutAsyncStoreEnabled() throws Exception { + checkRetry(Test.PUT_ASYNC, TestMemoryMode.HEAP, true); } /** * @throws Exception If failed. */ public void testInvoke() throws Exception { - checkRetry(Test.INVOKE); + checkRetry(Test.INVOKE, TestMemoryMode.HEAP, false); } /** * @throws Exception If failed. */ public void testInvokeAll() throws Exception { - checkRetry(Test.INVOKE_ALL); + checkRetry(Test.INVOKE_ALL, TestMemoryMode.HEAP, false); + } + + /** + * @throws Exception If failed. + */ + public void testInvokeAllOffheapSwap() throws Exception { + checkRetry(Test.INVOKE_ALL, TestMemoryMode.OFFHEAP_EVICT_SWAP, false); + } + + /** + * @throws Exception If failed. + */ + public void testInvokeAllOffheapTiered() throws Exception { + checkRetry(Test.INVOKE_ALL, TestMemoryMode.OFFHEAP_TIERED, false); } /** * @param test Test type. + * @param memMode Memory mode. + * @param store If {@code true} uses cache with store. * @throws Exception If failed. */ - private void checkRetry(Test test) throws Exception { + private void checkRetry(Test test, TestMemoryMode memMode, boolean store) throws Exception { + ignite(0).createCache(cacheConfiguration(memMode, store)); + final AtomicBoolean finished = new AtomicBoolean(); int keysCnt = keysCount(); - IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + IgniteInternalFuture fut = runAsync(new Callable() { @Override public Object call() throws Exception { while (!finished.get()) { stopGrid(3); @@ -155,10 +253,12 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr int iter = 0; try { - if (atomicityMode() == ATOMIC) - assertEquals(writeOrderMode(), cache.getConfiguration(CacheConfiguration.class).getAtomicWriteOrderMode()); + if (atomicityMode() == ATOMIC) { + assertEquals(writeOrderMode(), + cache.getConfiguration(CacheConfiguration.class).getAtomicWriteOrderMode()); + } - long stopTime = System.currentTimeMillis() + 60_000; + long stopTime = System.currentTimeMillis() + DURATION; switch (test) { case PUT: { @@ -288,7 +388,7 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr for (int i = 0; i < keysCnt; i++) assertEquals((Integer)iter, cache.get(i)); - for (int i = 0; i < gridCount(); i++) { + for (int i = 0; i < GRID_CNT; i++) { IgniteKernal ignite = (IgniteKernal)grid(i); Collection futs = ignite.context().cache().context().mvcc().atomicFutures(); @@ -316,9 +416,11 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr * @throws Exception If failed. */ private void checkFailsWithNoRetries(boolean async) throws Exception { + ignite(0).createCache(cacheConfiguration(TestMemoryMode.HEAP, false)); + final AtomicBoolean finished = new AtomicBoolean(); - IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + IgniteInternalFuture fut = runAsync(new Callable() { @Override public Object call() throws Exception { while (!finished.get()) { stopGrid(3); @@ -428,4 +530,26 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr return old == null ? 0 : old; } } + + /** + * + */ + private static class TestStoreFactory implements Factory { + /** {@inheritDoc} */ + @Override public CacheStore create() { + return new CacheStoreAdapter() { + @Override public Object load(Object key) throws CacheLoaderException { + return null; + } + + @Override public void write(Cache.Entry entry) throws CacheWriterException { + // No-op. + } + + @Override public void delete(Object key) throws CacheWriterException { + // No-op. + } + }; + } + } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAtomicSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAtomicSelfTest.java index 9d84609..3d7c7d7 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAtomicSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAtomicSelfTest.java @@ -47,6 +47,8 @@ public class IgniteCachePutRetryAtomicSelfTest extends IgniteCachePutRetryAbstra * @throws Exception If failed. */ public void testPutInsideTransaction() throws Exception { + ignite(0).createCache(cacheConfiguration(GridTestUtils.TestMemoryMode.HEAP, false)); + CacheConfiguration ccfg = new CacheConfiguration<>(); ccfg.setName("tx-cache"); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java index 7c66efc..f61faf2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java @@ -31,12 +31,13 @@ import org.apache.ignite.IgniteAtomicLong; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheEntryProcessor; -import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.testframework.GridTestUtils; import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; +import static org.apache.ignite.testframework.GridTestUtils.TestMemoryMode; +import static org.apache.ignite.testframework.GridTestUtils.runAsync; +import static org.apache.ignite.testframework.GridTestUtils.runMultiThreadedAsync; /** * @@ -50,11 +51,6 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr return TRANSACTIONAL; } - /** {@inheritDoc} */ - @Override protected NearCacheConfiguration nearConfiguration() { - return null; - } - /** * @throws Exception If failed. */ @@ -63,7 +59,7 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr IgniteAtomicLong atomic = ignite(0).atomicLong("TestAtomic", 0, true); - IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + IgniteInternalFuture fut = runAsync(new Callable() { @Override public Object call() throws Exception { while (!finished.get()) { stopGrid(3); @@ -92,15 +88,42 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr } } - /** {@inheritDoc} */ - @SuppressWarnings("unchecked") + /** + * @throws Exception If failed. + */ public void testExplicitTransactionRetries() throws Exception { + explicitTransactionRetries(TestMemoryMode.HEAP, false); + } + + /** + * @throws Exception If failed. + */ + public void testExplicitTransactionRetriesStoreEnabled() throws Exception { + explicitTransactionRetries(TestMemoryMode.HEAP, true); + } + + /** + * @throws Exception If failed. + */ + public void testExplicitTransactionRetriesOffheapSwap() throws Exception { + explicitTransactionRetries(TestMemoryMode.OFFHEAP_EVICT_SWAP, false); + } + + /** + * @param memMode Memory mode. + * @param store If {@code true} uses cache with store. + * @throws Exception If failed. + */ + @SuppressWarnings("unchecked") + public void explicitTransactionRetries(TestMemoryMode memMode, boolean store) throws Exception { + ignite(0).createCache(cacheConfiguration(memMode, store)); + final AtomicInteger idx = new AtomicInteger(); int threads = 8; final AtomicReferenceArray err = new AtomicReferenceArray<>(threads); - IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(new Callable() { + IgniteInternalFuture fut = runMultiThreadedAsync(new Callable() { @Override public Object call() throws Exception { int th = idx.getAndIncrement(); int base = th * FACTOR; @@ -115,8 +138,7 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr if (i > 0 && i % 500 == 0) info("Done: " + i); } - } - catch (Exception e) { + } catch (Exception e) { err.set(th, e); } @@ -142,7 +164,7 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr } // Verify contents of the cache. - for (int g = 0; g < gridCount(); g++) { + for (int g = 0; g < GRID_CNT; g++) { IgniteCache cache = ignite(g).cache(null); for (int th = 0; th < threads; th++) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java index c598e97..3913957 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java @@ -43,6 +43,7 @@ import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.managers.communication.GridIoMessage; import org.apache.ignite.internal.processors.cache.GridCacheAdapter; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.internal.util.typedef.internal.CU; @@ -65,6 +66,7 @@ import static org.apache.ignite.cache.CacheRebalanceMode.SYNC; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC; +import static org.apache.ignite.testframework.GridTestUtils.TestMemoryMode; /** * Test GridDhtInvalidPartitionException handling in ATOMIC cache during restarts. @@ -83,19 +85,26 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA /** Write sync. */ private CacheWriteSynchronizationMode writeSync; + /** Memory mode. */ + private TestMemoryMode memMode; + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER).setForceServerMode(true)); - cfg.setCacheConfiguration(cacheConfiguration()); + CacheConfiguration ccfg = cacheConfiguration(); + + cfg.setCacheConfiguration(ccfg); cfg.setCommunicationSpi(new DelayCommunicationSpi()); if (testClientNode() && getTestGridName(0).equals(gridName)) cfg.setClientMode(true); + GridTestUtils.setMemoryMode(cfg, ccfg, memMode, 100, 1024); + return cfg; } @@ -137,53 +146,99 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA * @throws Exception If failed. */ public void testClockFullSync() throws Exception { - checkRestarts(CLOCK, FULL_SYNC); + checkRestarts(CLOCK, FULL_SYNC, TestMemoryMode.HEAP); + } + + /** + * @throws Exception If failed. + */ + public void testClockFullSyncSwap() throws Exception { + checkRestarts(CLOCK, FULL_SYNC, TestMemoryMode.SWAP); + } + + /** + * @throws Exception If failed. + */ + public void testClockFullSyncOffheapTiered() throws Exception { + checkRestarts(CLOCK, FULL_SYNC, TestMemoryMode.OFFHEAP_TIERED); + } + + /** + * @throws Exception If failed. + */ + public void testClockFullSyncOffheapSwap() throws Exception { + checkRestarts(CLOCK, FULL_SYNC, TestMemoryMode.OFFHEAP_EVICT_SWAP); } /** * @throws Exception If failed. */ public void testClockPrimarySync() throws Exception { - checkRestarts(CLOCK, PRIMARY_SYNC); + checkRestarts(CLOCK, PRIMARY_SYNC, TestMemoryMode.HEAP); } /** * @throws Exception If failed. */ public void testClockFullAsync() throws Exception { - checkRestarts(CLOCK, FULL_ASYNC); + checkRestarts(CLOCK, FULL_ASYNC, TestMemoryMode.HEAP); } /** * @throws Exception If failed. */ public void testPrimaryFullSync() throws Exception { - checkRestarts(PRIMARY, FULL_SYNC); + checkRestarts(PRIMARY, FULL_SYNC, TestMemoryMode.HEAP); + } + + /** + * @throws Exception If failed. + */ + public void testPrimaryFullSyncSwap() throws Exception { + checkRestarts(PRIMARY, FULL_SYNC, TestMemoryMode.SWAP); + } + + /** + * @throws Exception If failed. + */ + public void testPrimaryFullSyncOffheapTiered() throws Exception { + checkRestarts(PRIMARY, FULL_SYNC, TestMemoryMode.OFFHEAP_TIERED); + } + + /** + * @throws Exception If failed. + */ + public void testPrimaryFullSyncOffheapSwap() throws Exception { + checkRestarts(PRIMARY, FULL_SYNC, TestMemoryMode.OFFHEAP_EVICT_SWAP); } /** * @throws Exception If failed. */ public void testPrimaryPrimarySync() throws Exception { - checkRestarts(PRIMARY, PRIMARY_SYNC); + checkRestarts(PRIMARY, PRIMARY_SYNC, TestMemoryMode.HEAP); } /** * @throws Exception If failed. */ public void testPrimaryFullAsync() throws Exception { - checkRestarts(PRIMARY, FULL_ASYNC); + checkRestarts(PRIMARY, FULL_ASYNC, TestMemoryMode.HEAP); } /** * @param writeOrder Write order to check. * @param writeSync Write synchronization mode to check. + * @param memMode Memory mode. * @throws Exception If failed. */ - private void checkRestarts(CacheAtomicWriteOrderMode writeOrder, CacheWriteSynchronizationMode writeSync) + private void checkRestarts(CacheAtomicWriteOrderMode writeOrder, + CacheWriteSynchronizationMode writeSync, + TestMemoryMode memMode) throws Exception { this.writeOrder = writeOrder; this.writeSync = writeSync; + this.memMode = memMode; final int gridCnt = 6; @@ -227,16 +282,16 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA for (int i = 0; i < gridCnt; i++) { ClusterNode locNode = grid(i).localNode(); - GridCacheAdapter c = ((IgniteKernal)grid(i)).internalCache(); + IgniteCache cache = grid(i).cache(null); - GridCacheEntryEx entry = c.peekEx(key); + Object val = cache.localPeek(key); if (affNodes.contains(locNode)) { - if (entry == null) + if (val == null) return false; } else - assertNull(entry); + assertNull(val); } it.remove(); @@ -323,7 +378,20 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA GridCacheAdapter c = ((IgniteKernal)grid(i)).internalCache(); - GridCacheEntryEx entry = c.peekEx(k); + GridCacheEntryEx entry = null; + + if (memMode == TestMemoryMode.HEAP) + entry = c.peekEx(k); + else { + try { + entry = c.entryEx(k); + + entry.unswap(); + } + catch (GridDhtInvalidPartitionException e) { + // Skip key. + } + } for (int r = 0; r < 10; r++) { try { @@ -383,7 +451,7 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA */ private static class DelayCommunicationSpi extends TcpCommunicationSpi { /** {@inheritDoc} */ - @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure ackClosure) + @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure ackC) throws IgniteSpiException { try { if (delayMessage((GridIoMessage)msg)) @@ -393,7 +461,7 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA throw new IgniteSpiException(e); } - super.sendMessage(node, msg, ackClosure); + super.sendMessage(node, msg, ackC); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicPrimaryWriteOrderRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicPrimaryWriteOrderRemoveFailureTest.java index 844bde3..2fd8a82 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicPrimaryWriteOrderRemoveFailureTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicPrimaryWriteOrderRemoveFailureTest.java @@ -17,9 +17,9 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.atomic; +import org.apache.ignite.cache.CacheAtomicWriteOrderMode; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; -import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.internal.processors.cache.GridCacheAbstractRemoveFailureTest; @@ -42,17 +42,12 @@ public class GridCacheAtomicPrimaryWriteOrderRemoveFailureTest extends GridCache } /** {@inheritDoc} */ - @Override protected NearCacheConfiguration nearConfiguration() { - return null; + @Override protected CacheAtomicWriteOrderMode atomicWriteOrderMode() { + return PRIMARY; } /** {@inheritDoc} */ - @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception { - CacheConfiguration cfg = super.cacheConfiguration(gridName); - - cfg.setBackups(1); - cfg.setAtomicWriteOrderMode(PRIMARY); - - return cfg; + @Override protected NearCacheConfiguration nearCache() { + return null; } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicRemoveFailureTest.java index 9966e18..a21cd35 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicRemoveFailureTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicRemoveFailureTest.java @@ -17,9 +17,9 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.atomic; +import org.apache.ignite.cache.CacheAtomicWriteOrderMode; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; -import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.internal.processors.cache.GridCacheAbstractRemoveFailureTest; @@ -42,17 +42,12 @@ public class GridCacheAtomicRemoveFailureTest extends GridCacheAbstractRemoveFai } /** {@inheritDoc} */ - @Override protected NearCacheConfiguration nearConfiguration() { - return null; + @Override protected CacheAtomicWriteOrderMode atomicWriteOrderMode() { + return CLOCK; } /** {@inheritDoc} */ - @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception { - CacheConfiguration cfg = super.cacheConfiguration(gridName); - - cfg.setBackups(1); - cfg.setAtomicWriteOrderMode(CLOCK); - - return cfg; + @Override protected NearCacheConfiguration nearCache() { + return null; } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearRemoveFailureTest.java index a6fc2b9..d9c6e01 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearRemoveFailureTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearRemoveFailureTest.java @@ -17,9 +17,9 @@ package org.apache.ignite.internal.processors.cache.distributed.near; +import org.apache.ignite.cache.CacheAtomicWriteOrderMode; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; -import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.internal.processors.cache.GridCacheAbstractRemoveFailureTest; @@ -41,13 +41,12 @@ public class GridCacheAtomicNearRemoveFailureTest extends GridCacheAbstractRemov } /** {@inheritDoc} */ - @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception { - CacheConfiguration cfg = super.cacheConfiguration(gridName); - - cfg.setNearConfiguration(new NearCacheConfiguration()); - cfg.setBackups(1); - cfg.setAtomicWriteOrderMode(CLOCK); + @Override protected CacheAtomicWriteOrderMode atomicWriteOrderMode() { + return CLOCK; + } - return cfg; + /** {@inheritDoc} */ + @Override protected NearCacheConfiguration nearCache() { + return new NearCacheConfiguration(); } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderNearRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderNearRemoveFailureTest.java index 6362bc8..f6e7e2f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderNearRemoveFailureTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderNearRemoveFailureTest.java @@ -17,9 +17,9 @@ package org.apache.ignite.internal.processors.cache.distributed.near; +import org.apache.ignite.cache.CacheAtomicWriteOrderMode; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; -import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.internal.processors.cache.GridCacheAbstractRemoveFailureTest; @@ -41,13 +41,12 @@ public class GridCacheAtomicPrimaryWriteOrderNearRemoveFailureTest extends GridC } /** {@inheritDoc} */ - @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception { - CacheConfiguration cfg = super.cacheConfiguration(gridName); - - cfg.setNearConfiguration(new NearCacheConfiguration()); - cfg.setBackups(1); - cfg.setAtomicWriteOrderMode(PRIMARY); + @Override protected CacheAtomicWriteOrderMode atomicWriteOrderMode() { + return PRIMARY; + } - return cfg; + /** {@inheritDoc} */ + @Override protected NearCacheConfiguration nearCache() { + return new NearCacheConfiguration(); } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearRemoveFailureTest.java index 1cf3da1..a7e8eb9 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearRemoveFailureTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearRemoveFailureTest.java @@ -17,11 +17,12 @@ package org.apache.ignite.internal.processors.cache.distributed.near; +import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; -import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.internal.processors.cache.GridCacheAbstractRemoveFailureTest; +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; import static org.apache.ignite.cache.CacheMode.PARTITIONED; /** @@ -34,12 +35,12 @@ public class GridCacheNearRemoveFailureTest extends GridCacheAbstractRemoveFailu } /** {@inheritDoc} */ - @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception { - CacheConfiguration cfg = super.cacheConfiguration(gridName); - - cfg.setNearConfiguration(new NearCacheConfiguration()); - cfg.setBackups(1); + @Override protected CacheAtomicityMode atomicityMode() { + return TRANSACTIONAL; + } - return cfg; + /** {@inheritDoc} */ + @Override protected NearCacheConfiguration nearCache() { + return new NearCacheConfiguration(); } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java index 265c185..684d6e4 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java @@ -36,6 +36,11 @@ public class GridCachePartitionedNodeRestartTest extends GridCacheAbstractNodeRe c.getTransactionConfiguration().setDefaultTxConcurrency(PESSIMISTIC); + return c; + } + + /** {@inheritDoc} */ + @Override protected CacheConfiguration cacheConfiguration() { CacheConfiguration cc = defaultCacheConfiguration(); cc.setName(CACHE_NAME); @@ -49,9 +54,7 @@ public class GridCachePartitionedNodeRestartTest extends GridCacheAbstractNodeRe cc.setAffinity(new RendezvousAffinityFunction(false, partitions)); cc.setBackups(backups); - c.setCacheConfiguration(cc); - - return c; + return cc; } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java index 0e81828..a458aa7 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java @@ -44,6 +44,11 @@ public class GridCachePartitionedOptimisticTxNodeRestartTest extends GridCacheAb c.getTransactionConfiguration().setDefaultTxConcurrency(OPTIMISTIC); + return c; + } + + /** {@inheritDoc} */ + @Override protected CacheConfiguration cacheConfiguration() { CacheConfiguration cc = defaultCacheConfiguration(); cc.setName(CACHE_NAME); @@ -57,9 +62,7 @@ public class GridCachePartitionedOptimisticTxNodeRestartTest extends GridCacheAb cc.setNearConfiguration(nearEnabled() ? new NearCacheConfiguration() : null); - c.setCacheConfiguration(cc); - - return c; + return cc; } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedNodeRestartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedNodeRestartSelfTest.java index 5be228d..7e3e7e5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedNodeRestartSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedNodeRestartSelfTest.java @@ -30,9 +30,7 @@ import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; */ public class GridCacheReplicatedNodeRestartSelfTest extends GridCacheAbstractNodeRestartSelfTest { /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - IgniteConfiguration c = super.getConfiguration(gridName); - + @Override protected CacheConfiguration cacheConfiguration() { CacheConfiguration cc = defaultCacheConfiguration(); cc.setNearConfiguration(null); @@ -51,9 +49,7 @@ public class GridCacheReplicatedNodeRestartSelfTest extends GridCacheAbstractNod cc.setRebalanceBatchSize(20); - c.setCacheConfiguration(cc); - - return c; + return cc; } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapAbstractSelfTest.java index 4bda4f4..4cda0c2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapAbstractSelfTest.java @@ -42,7 +42,7 @@ public abstract class GridOffHeapMapAbstractSelfTest extends GridCommonAbstractT private static final Random RAND = new Random(); /** Unsafe map. */ - private GridOffHeapMap map; + private GridOffHeapMap map; /** */ protected float load = 0.75f; @@ -86,7 +86,7 @@ public abstract class GridOffHeapMapAbstractSelfTest extends GridCommonAbstractT /** * @return New map. */ - protected abstract GridOffHeapMap newMap(); + protected abstract GridOffHeapMap newMap(); /** * @@ -551,6 +551,10 @@ public abstract class GridOffHeapMapAbstractSelfTest extends GridCommonAbstractT evictCnt.incrementAndGet(); } + + @Override public boolean removeEvicted() { + return true; + } }; map = newMap(); @@ -587,6 +591,10 @@ public abstract class GridOffHeapMapAbstractSelfTest extends GridCommonAbstractT @Override public void onEvict(int part, int hash, byte[] k, byte[] v) { evictCnt.incrementAndGet(); } + + @Override public boolean removeEvicted() { + return true; + } }; map = newMap(); @@ -622,6 +630,10 @@ public abstract class GridOffHeapMapAbstractSelfTest extends GridCommonAbstractT @Override public void onEvict(int part, int hash, byte[] k, byte[] v) { evictCnt.incrementAndGet(); } + + @Override public boolean removeEvicted() { + return true; + } }; map = newMap(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapPerformanceAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapPerformanceAbstractTest.java index d1a1b20..f7388e8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapPerformanceAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapPerformanceAbstractTest.java @@ -39,7 +39,7 @@ public abstract class GridOffHeapMapPerformanceAbstractTest extends GridCommonAb new HashMap<>(LOAD_CNT); /** Unsafe map. */ - private GridOffHeapMap map; + private GridOffHeapMap map; /** */ protected float load = 0.75f; @@ -91,7 +91,7 @@ public abstract class GridOffHeapMapPerformanceAbstractTest extends GridCommonAb /** * @return New map. */ - protected abstract GridOffHeapMap newMap(); + protected abstract GridOffHeapMap newMap(); /** * @param key Key. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java index 03fcd4a..c40c10f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java @@ -882,6 +882,10 @@ public abstract class GridOffHeapPartitionedMapAbstractSelfTest extends GridComm evictCnt.incrementAndGet(); } + + @Override public boolean removeEvicted() { + return true; + } }; map = newMap(); @@ -921,6 +925,10 @@ public abstract class GridOffHeapPartitionedMapAbstractSelfTest extends GridComm @Override public void onEvict(int part, int hash, byte[] k, byte[] v) { evictCnt.incrementAndGet(); } + + @Override public boolean removeEvicted() { + return true; + } }; map = newMap(); @@ -957,6 +965,10 @@ public abstract class GridOffHeapPartitionedMapAbstractSelfTest extends GridComm @Override public void onEvict(int part, int hash, byte[] k, byte[] v) { evictCnt.incrementAndGet(); } + + @Override public boolean removeEvicted() { + return true; + } }; map = newMap(); @@ -1009,6 +1021,10 @@ public abstract class GridOffHeapPartitionedMapAbstractSelfTest extends GridComm evicted.set(key); } + + @Override public boolean removeEvicted() { + return true; + } }; map = newMap(); @@ -1072,6 +1088,10 @@ public abstract class GridOffHeapPartitionedMapAbstractSelfTest extends GridComm evicted.set(key); } + + @Override public boolean removeEvicted() { + return true; + } }; map = newMap(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapPerformanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapPerformanceTest.java index e758246..58ad494 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapPerformanceTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapPerformanceTest.java @@ -26,7 +26,7 @@ import org.apache.ignite.internal.util.offheap.GridOffHeapMapPerformanceAbstract */ public class GridUnsafeMapPerformanceTest extends GridOffHeapMapPerformanceAbstractTest { /** {@inheritDoc} */ - @Override protected GridOffHeapMap newMap() { + @Override protected GridOffHeapMap newMap() { return GridOffHeapMapFactory.unsafeMap(concurrency, load, initCap, mem, lruStripes, evictClo); } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapSelfTest.java index 43fdb34..0e36f3d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapSelfTest.java @@ -26,7 +26,7 @@ import org.apache.ignite.internal.util.offheap.GridOffHeapMapFactory; */ public class GridUnsafeMapSelfTest extends GridOffHeapMapAbstractSelfTest { /** {@inheritDoc} */ - @Override protected GridOffHeapMap newMap() { + @Override protected GridOffHeapMap newMap() { return GridOffHeapMapFactory.unsafeMap(concurrency, load, initCap, mem, lruStripes, evictLsnr); } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapMapPerformanceAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapMapPerformanceAbstractTest.java index ed37306..4064482 100644 --- a/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapMapPerformanceAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapMapPerformanceAbstractTest.java @@ -42,7 +42,7 @@ public abstract class GridOffHeapMapPerformanceAbstractTest extends GridCommonAb new HashMap<>(LOAD_CNT); /** Unsafe map. */ - private GridOffHeapMap map; + private GridOffHeapMap map; /** */ protected float load = 0.75f; @@ -94,7 +94,7 @@ public abstract class GridOffHeapMapPerformanceAbstractTest extends GridCommonAb /** * @return New map. */ - protected abstract GridOffHeapMap newMap(); + protected abstract GridOffHeapMap newMap(); /** * @param key Key. diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/unsafe/GridUnsafeMapPerformanceTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/unsafe/GridUnsafeMapPerformanceTest.java index 1486a9c..af691b4 100644 --- a/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/unsafe/GridUnsafeMapPerformanceTest.java +++ b/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/unsafe/GridUnsafeMapPerformanceTest.java @@ -26,7 +26,7 @@ import org.apache.ignite.internal.util.offheap.GridOffHeapMapPerformanceAbstract */ public class GridUnsafeMapPerformanceTest extends GridOffHeapMapPerformanceAbstractTest { /** {@inheritDoc} */ - @Override protected GridOffHeapMap newMap() { + @Override protected GridOffHeapMap newMap() { return GridOffHeapMapFactory.unsafeMap(concurrency, load, initCap, mem, lruStripes, evictClo); } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java index b585a8f..edf7c52 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java @@ -59,7 +59,11 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.Ignition; +import org.apache.ignite.cache.CacheMemoryMode; +import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; @@ -83,6 +87,7 @@ import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi; import org.apache.ignite.ssl.SslContextFactory; import org.apache.ignite.testframework.config.GridTestProperties; import org.jetbrains.annotations.NotNull; @@ -1582,4 +1587,116 @@ public final class GridTestUtils { suite.addTestSuite(test); } + + /** + * Sets cache configuration parameters according to test memory mode. + * + * @param cfg Ignite configuration. + * @param ccfg Cache configuration. + * @param testMode Test memory mode. + * @param maxHeapCnt Maximum number of entries in heap (used if test mode involves eviction from heap). + * @param maxOffheapSize Maximum offheap memory size (used if test mode involves eviction from offheap to swap). + */ + public static void setMemoryMode(IgniteConfiguration cfg, CacheConfiguration ccfg, + TestMemoryMode testMode, + int maxHeapCnt, + long maxOffheapSize) { + assert testMode != null; + assert ccfg != null; + + CacheMemoryMode memMode; + boolean swap = false; + boolean evictionPlc = false; + long offheapMaxMem = -1L; + + switch (testMode) { + case HEAP: { + memMode = CacheMemoryMode.ONHEAP_TIERED; + swap = false; + + break; + } + + case SWAP: { + memMode = CacheMemoryMode.ONHEAP_TIERED; + evictionPlc = true; + swap = true; + + break; + } + + case OFFHEAP_TIERED: { + memMode = CacheMemoryMode.OFFHEAP_TIERED; + offheapMaxMem = 0; + + break; + } + + case OFFHEAP_TIERED_SWAP: { + assert maxOffheapSize > 0 : maxOffheapSize; + + memMode = CacheMemoryMode.OFFHEAP_TIERED; + offheapMaxMem = maxOffheapSize; + swap = true; + + break; + } + + case OFFHEAP_EVICT: { + memMode = CacheMemoryMode.ONHEAP_TIERED; + evictionPlc = true; + offheapMaxMem = 0; + + break; + } + + case OFFHEAP_EVICT_SWAP: { + assert maxOffheapSize > 0 : maxOffheapSize; + + memMode = CacheMemoryMode.ONHEAP_TIERED; + swap = true; + evictionPlc = true; + offheapMaxMem = maxOffheapSize; + + break; + } + + default: + throw new IllegalArgumentException("Invalid mode: " + testMode); + } + + ccfg.setMemoryMode(memMode); + ccfg.setSwapEnabled(swap); + + if (swap && cfg != null) + cfg.setSwapSpaceSpi(new FileSwapSpaceSpi()); + + if (evictionPlc) { + LruEvictionPolicy plc = new LruEvictionPolicy(); + + plc.setMaxSize(maxHeapCnt); + + ccfg.setEvictionPolicy(plc); + } + + ccfg.setOffHeapMaxMemory(offheapMaxMem); + } + + /** + * + */ + public enum TestMemoryMode { + /** Heap only. */ + HEAP, + /** Evict from heap to swap with eviction policy. */ + SWAP, + /** Always evict to offheap, no swap. */ + OFFHEAP_TIERED, + /** Always evict to offheap + evict from offheap to swap when max offheap memory limit is reached. */ + OFFHEAP_TIERED_SWAP, + /** Evict to offheap with eviction policy, no swap. */ + OFFHEAP_EVICT, + /** Evict to offheap with eviction policy + evict from offheap to swap when max offheap memory limit is reached. */ + OFFHEAP_EVICT_SWAP, + } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java index eaeb7b3..abc8765 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java @@ -91,12 +91,6 @@ public class IgniteCacheFailoverTestSuite extends TestSuite { suite.addTestSuite(IgniteCacheTxNearDisabledPutGetRestartTest.class); suite.addTestSuite(IgniteCacheTxNearDisabledFairAffinityPutGetRestartTest.class); - suite.addTestSuite(IgniteCachePutRetryAtomicSelfTest.class); - suite.addTestSuite(IgniteCachePutRetryAtomicPrimaryWriteOrderSelfTest.class); - suite.addTestSuite(IgniteCachePutRetryTransactionalSelfTest.class); - - suite.addTestSuite(IgniteCacheSslStartStopSelfTest.class); - suite.addTestSuite(IgniteCacheSizeFailoverTest.class); suite.addTestSuite(IgniteCacheTopologySafeGetSelfTest.class); diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite3.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite3.java new file mode 100644 index 0000000..318db9e --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite3.java @@ -0,0 +1,62 @@ +/* + * 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.testsuites; + +import junit.framework.TestSuite; +import org.apache.ignite.internal.processors.cache.GridCacheIncrementTransformTest; +import org.apache.ignite.internal.processors.cache.IgniteCacheTopologySafeGetSelfTest; +import org.apache.ignite.internal.processors.cache.distributed.*; +import org.apache.ignite.internal.processors.cache.distributed.dht.*; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.*; +import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicNearRemoveFailureTest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderNearRemoveFailureTest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearRemoveFailureTest; +import org.apache.ignite.spi.communication.tcp.IgniteCacheSslStartStopSelfTest; +import org.apache.ignite.testframework.GridTestUtils; + +import java.util.Set; + +/** + * Test suite. + */ +public class IgniteCacheFailoverTestSuite3 extends TestSuite { + /** + * @return Ignite Cache Failover test suite. + * @throws Exception Thrown in case of the failure. + */ + public static TestSuite suite() throws Exception { + return suite(null); + } + + /** + * @param ignoredTests Tests don't include in the execution. + * @return Test suite. + * @throws Exception Thrown in case of the failure. + */ + public static TestSuite suite(Set ignoredTests) throws Exception { + TestSuite suite = new TestSuite("Cache Failover Test Suite3"); + + suite.addTestSuite(IgniteCachePutRetryAtomicSelfTest.class); + suite.addTestSuite(IgniteCachePutRetryAtomicPrimaryWriteOrderSelfTest.class); + suite.addTestSuite(IgniteCachePutRetryTransactionalSelfTest.class); + + suite.addTestSuite(IgniteCacheSslStartStopSelfTest.class); + + return suite; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheRestartTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheRestartTestSuite.java index 869f25c..34ef79b 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheRestartTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheRestartTestSuite.java @@ -18,18 +18,13 @@ package org.apache.ignite.testsuites; import junit.framework.TestSuite; -import org.apache.ignite.internal.processors.cache.GridCachePutAllFailoverSelfTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheAtomicPutAllFailoverSelfTest; -import org.apache.ignite.internal.processors.cache.IgniteCachePutAllRestartTest; -import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheAtomicNodeRestartTest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledOptimisticTxNodeRestartTest; import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedNodeRestartTest; import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedOptimisticTxNodeRestartTest; import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedNodeRestartSelfTest; -import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheAtomicReplicatedNodeRestartSelfTest; /** - * In-Memory Data Grid stability test suite on changing topology. + * Cache stability test suite on changing topology. */ public class IgniteCacheRestartTestSuite extends TestSuite { /** @@ -44,13 +39,6 @@ public class IgniteCacheRestartTestSuite extends TestSuite { suite.addTestSuite(GridCacheReplicatedNodeRestartSelfTest.class); suite.addTestSuite(GridCachePartitionedNearDisabledOptimisticTxNodeRestartTest.class); - suite.addTestSuite(IgniteCacheAtomicNodeRestartTest.class); - suite.addTestSuite(IgniteCacheAtomicReplicatedNodeRestartSelfTest.class); - - suite.addTestSuite(IgniteCacheAtomicPutAllFailoverSelfTest.class); - suite.addTestSuite(IgniteCachePutAllRestartTest.class); - suite.addTestSuite(GridCachePutAllFailoverSelfTest.class); - return suite; } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheRestartTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheRestartTestSuite2.java new file mode 100644 index 0000000..c9e9467 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheRestartTestSuite2.java @@ -0,0 +1,47 @@ +/* + * 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.testsuites; + +import junit.framework.TestSuite; +import org.apache.ignite.internal.processors.cache.GridCachePutAllFailoverSelfTest; +import org.apache.ignite.internal.processors.cache.IgniteCacheAtomicPutAllFailoverSelfTest; +import org.apache.ignite.internal.processors.cache.IgniteCachePutAllRestartTest; +import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheAtomicNodeRestartTest; +import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheAtomicReplicatedNodeRestartSelfTest; + +/** + * Cache stability test suite on changing topology. + */ +public class IgniteCacheRestartTestSuite2 extends TestSuite { + /** + * @return Suite. + * @throws Exception If failed. + */ + public static TestSuite suite() throws Exception { + TestSuite suite = new TestSuite("Cache Restart Test Suite2"); + + suite.addTestSuite(IgniteCacheAtomicNodeRestartTest.class); + suite.addTestSuite(IgniteCacheAtomicReplicatedNodeRestartSelfTest.class); + + suite.addTestSuite(IgniteCacheAtomicPutAllFailoverSelfTest.class); + suite.addTestSuite(IgniteCachePutAllRestartTest.class); + suite.addTestSuite(GridCachePutAllFailoverSelfTest.class); + + return suite; + } +} diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2AbstractKeyValueRow.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2AbstractKeyValueRow.java index 63e2bc3..07c49a5 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2AbstractKeyValueRow.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2AbstractKeyValueRow.java @@ -182,9 +182,11 @@ public abstract class GridH2AbstractKeyValueRow extends GridH2Row { /** {@inheritDoc} */ @Override public Value getValue(int col) { if (col < DEFAULT_COLUMNS_COUNT) { - Value v = peekValue(col); + Value v; if (col == VAL_COL) { + v = syncValue(0); + long start = 0; int attempt = 0; @@ -206,11 +208,15 @@ public abstract class GridH2AbstractKeyValueRow extends GridH2Row { Object valObj = desc.readFromSwap(k); if (valObj != null) { - Value upd = desc.wrap(valObj, desc.valueType()); + // Even if we've found valObj in swap, it is may be some new value, + // while the needed value was already unswapped, so we have to recheck it. + if ((v = WeakValue.unwrap(syncValue(0))) == null && (v = getOffheapValue(VAL_COL)) == null) { + Value upd = desc.wrap(valObj, desc.valueType()); - v = updateWeakValue(upd); + v = updateWeakValue(upd); - return v == null ? upd : v; + return v == null ? upd : v; + } } else { // If nothing found in swap then we should be already unswapped. @@ -230,18 +236,21 @@ public abstract class GridH2AbstractKeyValueRow extends GridH2Row { ". This can happen due to a long GC pause."); } } - - if (v == null) { + else { assert col == KEY_COL : col; - v = getOffheapValue(KEY_COL); + v = peekValue(KEY_COL); - assert v != null : v; + if (v == null) { + v = getOffheapValue(KEY_COL); - setValue(KEY_COL, v); + assert v != null; - if (peekValue(VAL_COL) == null) - cache(); + setValue(KEY_COL, v); + + if (peekValue(VAL_COL) == null) + cache(); + } } assert !(v instanceof WeakValue) : v; diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java index 8f453a5..eb0880a 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java @@ -249,6 +249,10 @@ public class GridCacheOffHeapAndSwapSelfTest extends GridCommonAbstractTest { assertEquals(ENTRY_CNT - i - 1, cache.localSize(CachePeekMode.ONHEAP)); } + log.info("Cache size [onheap=" + cache.localSize(CachePeekMode.ONHEAP) + + ", offheap=" + cache.localSize(CachePeekMode.OFFHEAP) + + ", swap=" + cache.localSize(CachePeekMode.SWAP) + ']'); + // Ensure that part of entries located in off-heap memory and part is swapped. assertEquals(0, cache.localSize(CachePeekMode.ONHEAP)); assertTrue(cache.localSize(CachePeekMode.OFFHEAP) > 0); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java index b02b37e..be644e2 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java @@ -32,7 +32,6 @@ import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cache.CacheMemoryMode; import org.apache.ignite.cache.CachePeekMode; -import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy; import org.apache.ignite.cache.query.ScanQuery; import org.apache.ignite.cache.query.SqlQuery; @@ -55,6 +54,7 @@ import org.jetbrains.annotations.Nullable; import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; import static org.apache.ignite.cache.CacheMode.PARTITIONED; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; /** * Multi-threaded tests for cache queries. @@ -111,7 +111,7 @@ public class IgniteCacheQueryMultiThreadedSelfTest extends GridCommonAbstractTes cacheCfg.setCacheMode(PARTITIONED); cacheCfg.setAtomicityMode(TRANSACTIONAL); - cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + cacheCfg.setWriteSynchronizationMode(FULL_SYNC); cacheCfg.setSwapEnabled(true); cacheCfg.setBackups(1); @@ -139,6 +139,11 @@ public class IgniteCacheQueryMultiThreadedSelfTest extends GridCommonAbstractTes return cacheCfg; } + /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return DURATION + 60_000; + } + /** * */ diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.java index 909fd74..d7d2b5a 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.java @@ -22,11 +22,6 @@ package org.apache.ignite.internal.processors.cache; */ public class IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest extends IgniteCacheQueryOffheapMultiThreadedSelfTest { /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - fail("IGNITE-959"); - } - - /** {@inheritDoc} */ @Override protected boolean evictsEnabled() { return true; } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java index 7e72292..5dbb12c 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java @@ -315,7 +315,7 @@ public class IgniteCacheClientQueryReplicatedNodeRestartSelfTest extends GridCom info("Executed queries: " + c); } } - }, qryThreadNum); + }, qryThreadNum, "query-thread"); final AtomicInteger restartCnt = new AtomicInteger(); @@ -334,10 +334,14 @@ public class IgniteCacheClientQueryReplicatedNodeRestartSelfTest extends GridCom } while (!locks.compareAndSet(g, 0, -1)); + log.info("Stop node: " + g); + stopGrid(g); Thread.sleep(rnd.nextInt(nodeLifeTime)); + log.info("Start node: " + g); + startGrid(g); Thread.sleep(rnd.nextInt(nodeLifeTime)); @@ -352,7 +356,7 @@ public class IgniteCacheClientQueryReplicatedNodeRestartSelfTest extends GridCom return true; } - }, restartThreadsNum); + }, restartThreadsNum, "restart-thread"); Thread.sleep(duration); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest.java index 32da55a..45a249d 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest.java @@ -149,7 +149,7 @@ public class IgniteCacheQueryNodeRestartSelfTest extends GridCacheAbstractSelfTe info("Executed queries: " + c); } } - }, qryThreadNum); + }, qryThreadNum, "query-thread"); final AtomicInteger restartCnt = new AtomicInteger(); @@ -178,7 +178,7 @@ public class IgniteCacheQueryNodeRestartSelfTest extends GridCacheAbstractSelfTe return true; } - }, 1); + }, 1, "restart-thread"); Thread.sleep(duration); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java index 9e903d1..1276405 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java @@ -277,7 +277,7 @@ public class IgniteCacheQueryNodeRestartSelfTest2 extends GridCommonAbstractTest info("Executed queries: " + c); } } - }, qryThreadNum); + }, qryThreadNum, "query-thread"); final AtomicInteger restartCnt = new AtomicInteger(); @@ -296,10 +296,14 @@ public class IgniteCacheQueryNodeRestartSelfTest2 extends GridCommonAbstractTest } while (!locks.compareAndSet(g, 0, -1)); + log.info("Stop node: " + g); + stopGrid(g); Thread.sleep(rnd.nextInt(nodeLifeTime)); + log.info("Start node: " + g); + startGrid(g); Thread.sleep(rnd.nextInt(nodeLifeTime)); @@ -314,7 +318,7 @@ public class IgniteCacheQueryNodeRestartSelfTest2 extends GridCommonAbstractTest return true; } - }, restartThreadsNum); + }, restartThreadsNum, "restart-thread"); Thread.sleep(duration);