diff --git a/commons/all/src/main/java/org/infinispan/commons/reactive/RxJavaInterop.java b/commons/all/src/main/java/org/infinispan/commons/reactive/RxJavaInterop.java index d215c3693384..0f001307e537 100644 --- a/commons/all/src/main/java/org/infinispan/commons/reactive/RxJavaInterop.java +++ b/commons/all/src/main/java/org/infinispan/commons/reactive/RxJavaInterop.java @@ -8,6 +8,8 @@ import io.reactivex.rxjava3.core.Flowable; import io.reactivex.rxjava3.functions.Consumer; import io.reactivex.rxjava3.functions.Function; +import io.reactivex.rxjava3.processors.AsyncProcessor; +import io.reactivex.rxjava3.processors.FlowableProcessor; /** * Static factory class that provides methods to obtain commonly used instances for interoperation between RxJava @@ -54,9 +56,25 @@ public static Consumer emptyConsumer() { return (Consumer) emptyConsumer; } + /** + * Returns a {@link FlowableProcessor} that is already complete and will ignore any value submitted to it and will + * immediately cancel any subscriptions it receives. + * @return processor that is completed + * @param user value type + */ + public static FlowableProcessor completedFlowableProcessor() { + return (FlowableProcessor) completeFlowableProcessor; + } + private static final Function identityFunction = i -> i; private static final Consumer emptyConsumer = ignore -> { }; private static final Function, Object> entryToKeyFunction = Map.Entry::getKey; private static final Function, Object> entryToValueFunction = Map.Entry::getValue; private static final Function> wrapThrowable = t -> Flowable.error(Util.rewrapAsCacheException(t)); + private static final FlowableProcessor completeFlowableProcessor; + + static { + completeFlowableProcessor = AsyncProcessor.create(); + completeFlowableProcessor.onComplete(); + } } diff --git a/core/src/main/java/org/infinispan/commands/write/RemoveCommand.java b/core/src/main/java/org/infinispan/commands/write/RemoveCommand.java index edacc4f26ef1..046ec22de18f 100644 --- a/core/src/main/java/org/infinispan/commands/write/RemoveCommand.java +++ b/core/src/main/java/org/infinispan/commands/write/RemoveCommand.java @@ -55,6 +55,19 @@ public Object acceptVisitor(InvocationContext ctx, Visitor visitor) throws Throw return visitor.visitRemoveCommand(ctx, this); } + @Override + public boolean shouldReplicate(InvocationContext ctx, boolean requireReplicateIfRemote) { + if (!isSuccessful()) { + return false; + } + // XSITE backup should always replicate remove command + // If skip cache load is set we don't know if the store had a null value for remove so we have to replicate still + // Also if this is a backup write then we can't skip replication to stores + // Also if the caller says we must replicte on remote, make sure we are local + return (!nonExistent || hasAnyFlag(FlagBitSets.SKIP_XSITE_BACKUP | + FlagBitSets.SKIP_CACHE_LOAD) || (requireReplicateIfRemote && (ctx == null || !ctx.isOriginLocal()))); + } + @Override public byte getCommandId() { return COMMAND_ID; @@ -117,7 +130,7 @@ public boolean isConditional() { } public void nonExistant() { - nonExistent = false; + nonExistent = true; } public boolean isNonExistent() { diff --git a/core/src/main/java/org/infinispan/commands/write/RemoveExpiredCommand.java b/core/src/main/java/org/infinispan/commands/write/RemoveExpiredCommand.java index 6cc5f0c7b399..af9c3363273b 100644 --- a/core/src/main/java/org/infinispan/commands/write/RemoveExpiredCommand.java +++ b/core/src/main/java/org/infinispan/commands/write/RemoveExpiredCommand.java @@ -57,6 +57,12 @@ public byte getCommandId() { return COMMAND_ID; } + @Override + public boolean shouldReplicate(InvocationContext ctx, boolean requireReplicateIfRemote) { + // TODO: I think expiration always has to replicate - check if works later + return isSuccessful(); + } + @Override public String toString() { return "RemoveExpiredCommand{" + diff --git a/core/src/main/java/org/infinispan/commands/write/WriteCommand.java b/core/src/main/java/org/infinispan/commands/write/WriteCommand.java index 5452c49028e0..6168239d73e7 100644 --- a/core/src/main/java/org/infinispan/commands/write/WriteCommand.java +++ b/core/src/main/java/org/infinispan/commands/write/WriteCommand.java @@ -6,6 +6,7 @@ import org.infinispan.commands.FlagAffectedCommand; import org.infinispan.commands.TopologyAffectedCommand; import org.infinispan.commands.VisitableCommand; +import org.infinispan.context.InvocationContext; import org.infinispan.metadata.impl.PrivateMetadata; /** @@ -24,6 +25,19 @@ public interface WriteCommand extends VisitableCommand, FlagAffectedCommand, Top */ boolean isSuccessful(); + /** + * Some commands may be successful but not need to be replicated to other nodes, stores or listeners. For example + * a unconditional remove may be performed on a key that doesn't exist. In that case the command is still successful + * but does not need to replicate that information other places. + * @param ctx invocation context if present, may be null + * @param requireReplicateIfRemote if the command can replicate even if not a locally invoked command + * @return whether the command should replicate + * @implSpec default just invokes {@link #isSuccessful()} + */ + default boolean shouldReplicate(InvocationContext ctx, boolean requireReplicateIfRemote) { + return isSuccessful(); + } + /** * Certain commands only work based on a certain condition or state of the cache. For example, {@link * org.infinispan.Cache#putIfAbsent(Object, Object)} only does anything if a condition is met, i.e., the entry in diff --git a/core/src/main/java/org/infinispan/configuration/parsing/CacheParser.java b/core/src/main/java/org/infinispan/configuration/parsing/CacheParser.java index 0e8c06698a1f..2d33a2a5f019 100644 --- a/core/src/main/java/org/infinispan/configuration/parsing/CacheParser.java +++ b/core/src/main/java/org/infinispan/configuration/parsing/CacheParser.java @@ -1369,7 +1369,7 @@ private void parseIndex(ConfigurationReader reader, SoftIndexFileStoreConfigurat break; } case SEGMENTS: - builder.indexSegments(ParseUtils.parseInt(reader, i, value)); + ignoreAttribute(reader, i); break; case INDEX_QUEUE_LENGTH: builder.indexQueueLength(ParseUtils.parseInt(reader, i, value)); diff --git a/core/src/main/java/org/infinispan/container/DataContainer.java b/core/src/main/java/org/infinispan/container/DataContainer.java index 5374dd996d1b..bea905553bc2 100644 --- a/core/src/main/java/org/infinispan/container/DataContainer.java +++ b/core/src/main/java/org/infinispan/container/DataContainer.java @@ -6,7 +6,6 @@ import org.infinispan.container.entries.InternalCacheEntry; import org.infinispan.container.impl.InternalEntryFactory; -import org.infinispan.eviction.impl.ActivationManager; import org.infinispan.eviction.impl.PassivationManager; import org.infinispan.factories.scopes.Scope; import org.infinispan.factories.scopes.Scopes; @@ -59,9 +58,6 @@ public interface DataContainer extends Iterable> /** * Puts an entry in the cache along with metadata adding information such lifespan of entry, max idle time, version * information...etc. - *

- * The {@code key} must be activate by invoking {@link ActivationManager#activateAsync(Object, int)} - * boolean)}. * * @param k key under which to store entry * @param v value to store @@ -79,8 +75,6 @@ public interface DataContainer extends Iterable> /** * Removes an entry from the cache - *

- * The {@code key} must be activate by invoking {@link ActivationManager#activateAsync(Object, int)} * * @param k key to remove * @return entry removed, or null if it didn't exist or had expired @@ -127,8 +121,6 @@ default int size() { *

* See {@link org.infinispan.container.DataContainer.ComputeAction#compute(Object, * org.infinispan.container.entries.InternalCacheEntry, InternalEntryFactory)}. - *

- * The {@code key} must be activated by invoking {@link ActivationManager#activateAsync(Object, int)}. *

* Note the entry provided to {@link org.infinispan.container.DataContainer.ComputeAction} may be expired as these * entries are not filtered as many other methods do. diff --git a/core/src/main/java/org/infinispan/interceptors/distribution/BaseDistributionInterceptor.java b/core/src/main/java/org/infinispan/interceptors/distribution/BaseDistributionInterceptor.java index 60887a7393a0..c26849e434c0 100644 --- a/core/src/main/java/org/infinispan/interceptors/distribution/BaseDistributionInterceptor.java +++ b/core/src/main/java/org/infinispan/interceptors/distribution/BaseDistributionInterceptor.java @@ -233,6 +233,10 @@ protected Object primaryReturnHandler(InvocationContext ctx, AbstractDataWriteCo if (log.isTraceEnabled()) log.tracef("Skipping the replication of the conditional command as it did not succeed on primary owner (%s).", command); return localResult; } + if (!command.shouldReplicate(ctx, false)) { + if (log.isTraceEnabled()) log.tracef("Skipping the replication of the command as it does not need to be (%s).", command); + return localResult; + } LocalizedCacheTopology cacheTopology = CacheTopologyUtil.checkTopology(command, getCacheTopology()); int segment = SegmentSpecificCommand.extractSegment(command, command.getKey(), keyPartitioner); DistributionInfo distributionInfo = cacheTopology.getSegmentDistribution(segment); diff --git a/core/src/main/java/org/infinispan/interceptors/distribution/L1LastChanceInterceptor.java b/core/src/main/java/org/infinispan/interceptors/distribution/L1LastChanceInterceptor.java index 1eddca7ccfed..30aabaed95ca 100644 --- a/core/src/main/java/org/infinispan/interceptors/distribution/L1LastChanceInterceptor.java +++ b/core/src/main/java/org/infinispan/interceptors/distribution/L1LastChanceInterceptor.java @@ -25,6 +25,7 @@ import org.infinispan.commands.write.RemoveCommand; import org.infinispan.commands.write.ReplaceCommand; import org.infinispan.commands.write.WriteCommand; +import org.infinispan.commons.util.concurrent.CompletableFutures; import org.infinispan.context.InvocationContext; import org.infinispan.context.impl.FlagBitSets; import org.infinispan.context.impl.TxInvocationContext; @@ -34,7 +35,6 @@ import org.infinispan.interceptors.InvocationSuccessFunction; import org.infinispan.interceptors.impl.BaseRpcInterceptor; import org.infinispan.interceptors.locking.ClusteringDependentLogic; -import org.infinispan.commons.util.concurrent.CompletableFutures; import org.infinispan.util.logging.Log; import org.infinispan.util.logging.LogFactory; @@ -143,7 +143,7 @@ public Object visitDataWriteCommand(InvocationContext ctx, DataWriteCommand comm private Object handleDataWriteCommand(InvocationContext rCtx, DataWriteCommand writeCommand, Object rv, boolean assumeOriginKeptEntryInL1) { Object key; Object key1 = (key = writeCommand.getKey()); - if (shouldUpdateOnWriteCommand(writeCommand) && writeCommand.isSuccessful() && + if (shouldUpdateOnWriteCommand(writeCommand) && writeCommand.shouldReplicate(rCtx, true) && cdl.getCacheTopology().isWriteOwner(key1)) { if (log.isTraceEnabled()) { log.trace("Sending additional invalidation for requestors if necessary."); diff --git a/core/src/main/java/org/infinispan/interceptors/distribution/TriangleDistributionInterceptor.java b/core/src/main/java/org/infinispan/interceptors/distribution/TriangleDistributionInterceptor.java index f6eaf9a300fe..41d630275612 100644 --- a/core/src/main/java/org/infinispan/interceptors/distribution/TriangleDistributionInterceptor.java +++ b/core/src/main/java/org/infinispan/interceptors/distribution/TriangleDistributionInterceptor.java @@ -447,7 +447,7 @@ private Object localPrimaryOwnerWrite(InvocationCon final C dwCommand = (C) rCommand; final CommandInvocationId id = dwCommand.getCommandInvocationId(); Collection

backupOwners = distributionInfo.writeBackups(); - if (!dwCommand.isSuccessful() || backupOwners.isEmpty()) { + if (!dwCommand.shouldReplicate(rCtx, true) || backupOwners.isEmpty()) { if (log.isTraceEnabled()) { log.tracef("Not sending command %s to backups", id); } @@ -496,9 +496,11 @@ private Object remotePrimaryOwnerWrite(InvocationCo final C dwCommand = (C) rCommand; final CommandInvocationId id = dwCommand.getCommandInvocationId(); Collection
backupOwners = distributionInfo.writeBackups(); + // Note we have to replicate even if the command says not to with triangle if the command + // didn't originate at the primary since the backup may be the originator if (!dwCommand.isSuccessful() || backupOwners.isEmpty()) { if (log.isTraceEnabled()) { - log.tracef("Command %s not successful in primary owner.", id); + log.tracef("Command %s not replicating from primary owner.", id); } return rv; } diff --git a/core/src/main/java/org/infinispan/interceptors/impl/AbstractIracLocalSiteInterceptor.java b/core/src/main/java/org/infinispan/interceptors/impl/AbstractIracLocalSiteInterceptor.java index 7cf7a42a87b5..a4e6edd20610 100644 --- a/core/src/main/java/org/infinispan/interceptors/impl/AbstractIracLocalSiteInterceptor.java +++ b/core/src/main/java/org/infinispan/interceptors/impl/AbstractIracLocalSiteInterceptor.java @@ -192,7 +192,7 @@ protected void visitNonTxKey(InvocationContext ctx, Object key, WriteCommand com @SuppressWarnings("unused") private void handleNonTxDataWriteCommand(InvocationContext ctx, DataWriteCommand command, Object rv, Throwable t) { final Object key = command.getKey(); - if (!command.isSuccessful() || skipEntryCommit(ctx, command, key)) { + if (!command.shouldReplicate(ctx, true) || skipEntryCommit(ctx, command, key)) { return; } setMetadataToCacheEntry(ctx.lookupEntry(key), command.getSegment(), command.getInternalMetadata(key).iracMetadata()); diff --git a/core/src/main/java/org/infinispan/interceptors/impl/CacheWriterInterceptor.java b/core/src/main/java/org/infinispan/interceptors/impl/CacheWriterInterceptor.java index 3b3f8a682047..b6dc7f446fb3 100644 --- a/core/src/main/java/org/infinispan/interceptors/impl/CacheWriterInterceptor.java +++ b/core/src/main/java/org/infinispan/interceptors/impl/CacheWriterInterceptor.java @@ -37,6 +37,7 @@ import org.infinispan.commands.write.RemoveCommand; import org.infinispan.commands.write.ReplaceCommand; import org.infinispan.commands.write.WriteCommand; +import org.infinispan.commons.util.concurrent.CompletableFutures; import org.infinispan.configuration.cache.StoreConfiguration; import org.infinispan.container.entries.CacheEntry; import org.infinispan.container.entries.InternalCacheValue; @@ -60,7 +61,6 @@ import org.infinispan.transaction.impl.AbstractCacheTransaction; import org.infinispan.transaction.xa.GlobalTransaction; import org.infinispan.util.concurrent.AggregateCompletionStage; -import org.infinispan.commons.util.concurrent.CompletableFutures; import org.infinispan.util.concurrent.CompletionStages; import org.infinispan.util.logging.Log; import org.infinispan.util.logging.LogFactory; @@ -175,10 +175,14 @@ private Transaction suspendRunningTx(TxInvocationContext ctx) throws SystemEx return xaTx; } + boolean shouldReplicateRemove(InvocationContext ctx, RemoveCommand removeCommand) { + return removeCommand.shouldReplicate(ctx, true); + } + @Override public Object visitRemoveCommand(InvocationContext ctx, RemoveCommand command) throws Throwable { return invokeNextThenApply(ctx, command, (rCtx, removeCommand, rv) -> { - if (!isStoreEnabled(removeCommand) || rCtx.isInTxScope() || !removeCommand.isSuccessful() || + if (!isStoreEnabled(removeCommand) || rCtx.isInTxScope() || !shouldReplicateRemove(ctx, command) || !isProperWriter(rCtx, removeCommand, removeCommand.getKey())) { return rv; } @@ -220,7 +224,7 @@ public Object visitIracPutKeyValueCommand(InvocationContext ctx, IracPutKeyValue @Override public Object visitComputeCommand(InvocationContext ctx, ComputeCommand command) throws Throwable { return invokeNextThenApply(ctx, command, (rCtx, computeCommand, rv) -> { - if (!isStoreEnabled(computeCommand) || rCtx.isInTxScope() || !computeCommand.isSuccessful() || + if (!isStoreEnabled(computeCommand) || rCtx.isInTxScope() || !computeCommand.shouldReplicate(ctx, true) || !isProperWriter(rCtx, computeCommand, computeCommand.getKey())) return rv; @@ -244,7 +248,7 @@ public Object visitComputeCommand(InvocationContext ctx, ComputeCommand command) @Override public Object visitComputeIfAbsentCommand(InvocationContext ctx, ComputeIfAbsentCommand command) throws Throwable { return invokeNextThenApply(ctx, command, (rCtx, computeIfAbsentCommand, rv) -> { - if (!isStoreEnabled(computeIfAbsentCommand) || rCtx.isInTxScope() || !computeIfAbsentCommand.isSuccessful()) + if (!isStoreEnabled(computeIfAbsentCommand) || rCtx.isInTxScope() || !computeIfAbsentCommand.shouldReplicate(ctx, true)) return rv; if (!isProperWriter(rCtx, computeIfAbsentCommand, computeIfAbsentCommand.getKey())) return rv; @@ -301,7 +305,7 @@ public Object visitWriteOnlyKeyValueCommand(InvocationContext ctx, WriteOnlyKeyV private Object visitWriteCommand(InvocationContext ctx, T command) { return invokeNextThenApply(ctx, command, (rCtx, dataWriteCommand, rv) -> { - if (!isStoreEnabled(dataWriteCommand) || rCtx.isInTxScope() || !dataWriteCommand.isSuccessful() || + if (!isStoreEnabled(dataWriteCommand) || rCtx.isInTxScope() || !dataWriteCommand.shouldReplicate(ctx, true) || !isProperWriter(rCtx, dataWriteCommand, dataWriteCommand.getKey())) return rv; @@ -409,8 +413,8 @@ private Object visitWriteManyComman }); } - protected final InvocationStage store(TxInvocationContext ctx) throws Throwable { - CompletionStage batchStage = persistenceManager.performBatch(ctx, ((writeCommand, o) -> isProperWriter(ctx, writeCommand, o))); + protected InvocationStage store(TxInvocationContext ctx) throws Throwable { + CompletionStage batchStage = persistenceManager.performBatch(ctx, ((writeCommand, k, v) -> isProperWriter(ctx, writeCommand, k))); if (getStatisticsEnabled()) { batchStage.thenAccept(cacheStores::addAndGet); } @@ -490,7 +494,7 @@ protected boolean skipSharedStores(InvocationContext ctx, Object key, FlagAffect private Object visitDataWriteCommandToStore(InvocationContext ctx, DataWriteCommand command) { return invokeNextThenApply(ctx, command, (rCtx, cmd, rv) -> { - if (!isStoreEnabled(cmd) || rCtx.isInTxScope() || !cmd.isSuccessful()) + if (!isStoreEnabled(cmd) || rCtx.isInTxScope() || !cmd.shouldReplicate(ctx, true)) return rv; if (!isProperWriter(rCtx, cmd, cmd.getKey())) return rv; diff --git a/core/src/main/java/org/infinispan/interceptors/impl/DistCacheWriterInterceptor.java b/core/src/main/java/org/infinispan/interceptors/impl/DistCacheWriterInterceptor.java index 7fd2b6b88a6d..c7a2fe2e2143 100644 --- a/core/src/main/java/org/infinispan/interceptors/impl/DistCacheWriterInterceptor.java +++ b/core/src/main/java/org/infinispan/interceptors/impl/DistCacheWriterInterceptor.java @@ -19,7 +19,6 @@ import org.infinispan.distribution.DistributionInfo; import org.infinispan.distribution.DistributionManager; import org.infinispan.factories.annotations.Inject; -import org.infinispan.commons.util.concurrent.CompletableFutures; import org.infinispan.util.logging.Log; import org.infinispan.util.logging.LogFactory; @@ -62,7 +61,7 @@ protected void start() { public Object visitPutKeyValueCommand(InvocationContext ctx, PutKeyValueCommand command) throws Throwable { return invokeNextThenApply(ctx, command, (rCtx, putKeyValueCommand, rv) -> { Object key = putKeyValueCommand.getKey(); - if (!putKeyValueCommand.hasAnyFlag(FlagBitSets.ROLLING_UPGRADE) && (!isStoreEnabled(putKeyValueCommand) || rCtx.isInTxScope() || !putKeyValueCommand.isSuccessful())) + if (!putKeyValueCommand.hasAnyFlag(FlagBitSets.ROLLING_UPGRADE) && (!isStoreEnabled(putKeyValueCommand) || rCtx.isInTxScope() || !putKeyValueCommand.shouldReplicate(ctx, true))) return rv; if (!isProperWriter(rCtx, putKeyValueCommand, putKeyValueCommand.getKey())) return rv; @@ -75,7 +74,7 @@ public Object visitPutKeyValueCommand(InvocationContext ctx, PutKeyValueCommand public Object visitIracPutKeyValueCommand(InvocationContext ctx, IracPutKeyValueCommand command) { return invokeNextThenApply(ctx, command, (rCtx, cmd, rv) -> { Object key = cmd.getKey(); - if (!isStoreEnabled(cmd) || !cmd.isSuccessful()) + if (!isStoreEnabled(cmd) || !cmd.shouldReplicate(rCtx, true)) return rv; if (!isProperWriter(rCtx, cmd, cmd.getKey())) return rv; @@ -111,7 +110,7 @@ private boolean skipNonPrimary(InvocationContext rCtx, Object key, PutMapCommand public Object visitRemoveCommand(InvocationContext ctx, RemoveCommand command) throws Throwable { return invokeNextThenApply(ctx, command, (rCtx, removeCommand, rv) -> { Object key = removeCommand.getKey(); - if (!isStoreEnabled(removeCommand) || rCtx.isInTxScope() || !removeCommand.isSuccessful()) + if (!isStoreEnabled(removeCommand) || rCtx.isInTxScope() || !removeCommand.shouldReplicate(ctx, true)) return rv; if (!isProperWriter(rCtx, removeCommand, key)) return rv; @@ -130,7 +129,7 @@ public Object visitRemoveCommand(InvocationContext ctx, RemoveCommand command) t public Object visitReplaceCommand(InvocationContext ctx, ReplaceCommand command) throws Throwable { return invokeNextThenApply(ctx, command, (rCtx, replaceCommand, rv) -> { Object key = replaceCommand.getKey(); - if (!isStoreEnabled(replaceCommand) || rCtx.isInTxScope() || !replaceCommand.isSuccessful()) + if (!isStoreEnabled(replaceCommand) || rCtx.isInTxScope() || !replaceCommand.shouldReplicate(ctx, true)) return rv; if (!isProperWriter(rCtx, replaceCommand, replaceCommand.getKey())) return rv; @@ -143,23 +142,21 @@ public Object visitReplaceCommand(InvocationContext ctx, ReplaceCommand command) public Object visitComputeCommand(InvocationContext ctx, ComputeCommand command) throws Throwable { return invokeNextThenApply(ctx, command, (rCtx, computeCommand, rv) -> { Object key = computeCommand.getKey(); - if (!isStoreEnabled(computeCommand) || rCtx.isInTxScope() || !computeCommand.isSuccessful()) + if (!isStoreEnabled(computeCommand) || rCtx.isInTxScope() || !computeCommand.shouldReplicate(ctx, true)) return rv; if (!isProperWriter(rCtx, computeCommand, computeCommand.getKey())) return rv; CompletionStage stage; - if (computeCommand.isSuccessful() && rv == null) { + if (rv == null) { stage = persistenceManager.deleteFromAllStores(key, computeCommand.getSegment(), skipSharedStores(rCtx, key, computeCommand) ? PRIVATE : BOTH); if (log.isTraceEnabled()) { stage = stage.thenAccept(removed -> getLog().tracef("Removed entry under key %s and got response %s from CacheStore", key, removed)); } - } else if (computeCommand.isSuccessful()) { - stage = storeEntry(rCtx, key, computeCommand); } else { - stage = CompletableFutures.completedNull(); + stage = storeEntry(rCtx, key, computeCommand); } return delayedValue(stage, rv); }); @@ -169,7 +166,7 @@ public Object visitComputeCommand(InvocationContext ctx, ComputeCommand command) public Object visitComputeIfAbsentCommand(InvocationContext ctx, ComputeIfAbsentCommand command) throws Throwable { return invokeNextThenApply(ctx, command, (rCtx, computeIfAbsentCommand, rv) -> { Object key = computeIfAbsentCommand.getKey(); - if (!isStoreEnabled(computeIfAbsentCommand) || rCtx.isInTxScope() || !computeIfAbsentCommand.isSuccessful()) + if (!isStoreEnabled(computeIfAbsentCommand) || rCtx.isInTxScope() || !computeIfAbsentCommand.shouldReplicate(ctx, true)) return rv; if (!isProperWriter(rCtx, computeIfAbsentCommand, computeIfAbsentCommand.getKey())) return rv; diff --git a/core/src/main/java/org/infinispan/interceptors/impl/InvalidationInterceptor.java b/core/src/main/java/org/infinispan/interceptors/impl/InvalidationInterceptor.java index 7856d53461ff..04daa33467e7 100644 --- a/core/src/main/java/org/infinispan/interceptors/impl/InvalidationInterceptor.java +++ b/core/src/main/java/org/infinispan/interceptors/impl/InvalidationInterceptor.java @@ -25,6 +25,7 @@ import org.infinispan.commands.write.ReplaceCommand; import org.infinispan.commands.write.WriteCommand; import org.infinispan.commons.util.EnumUtil; +import org.infinispan.commons.util.concurrent.CompletableFutures; import org.infinispan.context.InvocationContext; import org.infinispan.context.impl.FlagBitSets; import org.infinispan.context.impl.LocalTxInvocationContext; @@ -41,7 +42,6 @@ import org.infinispan.jmx.annotations.Parameter; import org.infinispan.remoting.inboundhandler.DeliverOrder; import org.infinispan.remoting.transport.impl.VoidResponseCollector; -import org.infinispan.commons.util.concurrent.CompletableFutures; import org.infinispan.util.logging.Log; import org.infinispan.util.logging.LogFactory; @@ -231,6 +231,8 @@ private Object handleInvalidate(InvocationContext ctx, WriteCommand command, Obj return invokeNext(ctx, command); } return invokeNextThenApply(ctx, command, (rCtx, writeCommand, rv) -> { + // Invaldation ignores a commands attempt at avoiding replication (e.g. writeCommand.shouldReplicate) + // as the local node may not have the key but others could if (writeCommand.isSuccessful()) { if (keys != null && keys.length != 0) { if (!isLocalModeForced(writeCommand)) { diff --git a/core/src/main/java/org/infinispan/interceptors/impl/NonTxIracLocalSiteInterceptor.java b/core/src/main/java/org/infinispan/interceptors/impl/NonTxIracLocalSiteInterceptor.java index fa272cd68cfe..ab9e9536c1c3 100644 --- a/core/src/main/java/org/infinispan/interceptors/impl/NonTxIracLocalSiteInterceptor.java +++ b/core/src/main/java/org/infinispan/interceptors/impl/NonTxIracLocalSiteInterceptor.java @@ -153,7 +153,8 @@ private Object visitWriteCommand(InvocationContext ctx, WriteCommand command) { */ @SuppressWarnings("unused") private void handleWriteCommand(InvocationContext ctx, WriteCommand command, Object rv, Throwable t) { - if (!command.isSuccessful()) { + // TODO: this is another to look into + if (!command.shouldReplicate(ctx, true)) { return; } for (Object key : command.getAffectedKeys()) { diff --git a/core/src/main/java/org/infinispan/interceptors/impl/PassivationCacheLoaderInterceptor.java b/core/src/main/java/org/infinispan/interceptors/impl/PassivationCacheLoaderInterceptor.java index 6e78b0ebf0e6..75e495bf8fa5 100644 --- a/core/src/main/java/org/infinispan/interceptors/impl/PassivationCacheLoaderInterceptor.java +++ b/core/src/main/java/org/infinispan/interceptors/impl/PassivationCacheLoaderInterceptor.java @@ -7,7 +7,6 @@ import org.infinispan.commands.FlagAffectedCommand; import org.infinispan.container.entries.InternalCacheEntry; import org.infinispan.context.InvocationContext; -import org.infinispan.eviction.impl.ActivationManager; import org.infinispan.factories.annotations.Inject; import org.infinispan.util.concurrent.CompletionStages; import org.infinispan.util.concurrent.DataOperationOrderer; @@ -19,8 +18,11 @@ public class PassivationCacheLoaderInterceptor extends CacheLoaderIntercep private static final Log log = LogFactory.getLog(MethodHandles.lookup().lookupClass()); @Inject DataOperationOrderer orderer; - @Inject ActivationManager activationManager; + // Normally the data container/store is updated in the order of updating the store and then the data container. + // In doing so loading from the store and saving into the container is usually safe. + // However, passivation with a concurrent remove can cause an issue with a read where the result is the + // read value from the loader can resurrect the value into memory, thus we have to order the read to properly handle this case @Override public CompletionStage> loadAndStoreInDataContainer(InvocationContext ctx, Object key, int segment, FlagAffectedCommand cmd) { @@ -29,41 +31,11 @@ public CompletionStage> loadAndStoreInDataContainer(Inv CompletionStage> retrievalStage; if (delayStage != null && !CompletionStages.isCompletedSuccessfully(delayStage)) { + log.tracef("Found concurrent operation on key %s when attempting to load from store, waiting for its completion", key); retrievalStage = delayStage.thenCompose(ignore -> super.loadAndStoreInDataContainer(ctx, key, segment, cmd)); } else { retrievalStage = super.loadAndStoreInDataContainer(ctx, key, segment, cmd); } - if (CompletionStages.isCompletedSuccessfully(retrievalStage)) { - InternalCacheEntry ice = CompletionStages.join(retrievalStage); - activateAfterLoad(key, segment, orderer, activationManager, future, ice, null); - return retrievalStage; - } else { - return retrievalStage.whenComplete((value, t) -> { - activateAfterLoad(key, segment, orderer, activationManager, future, value, t); - }); - } - } - - static void activateAfterLoad(Object key, int segment, DataOperationOrderer orderer, ActivationManager activationManager, CompletableFuture future, InternalCacheEntry value, Throwable t) { - if (value != null) { - if (log.isTraceEnabled()) { - log.tracef("Activating key: %s - not waiting for response", value.getKey()); - } - // Note we don't wait on this to be removed, which allows the load to continue ahead. - // However, we can't release the orderer acquisition until the remove is complete - CompletionStage activationStage = activationManager.activateAsync(value.getKey(), segment); - if (!CompletionStages.isCompletedSuccessfully(activationStage)) { - activationStage.whenComplete((ignore, throwable) -> { - if (throwable != null) { - log.warnf("Activation of key %s failed for some reason", t); - } - orderer.completeOperation(key, future, Operation.READ); - }); - } else { - orderer.completeOperation(key, future, Operation.READ); - } - } else { - orderer.completeOperation(key, future, Operation.READ); - } + return retrievalStage.whenComplete((v, t) -> orderer.completeOperation(key, future, Operation.READ)); } } diff --git a/core/src/main/java/org/infinispan/interceptors/impl/PassivationClusteredCacheLoaderInterceptor.java b/core/src/main/java/org/infinispan/interceptors/impl/PassivationClusteredCacheLoaderInterceptor.java index 2f63a2119475..d21cd1e717c1 100644 --- a/core/src/main/java/org/infinispan/interceptors/impl/PassivationClusteredCacheLoaderInterceptor.java +++ b/core/src/main/java/org/infinispan/interceptors/impl/PassivationClusteredCacheLoaderInterceptor.java @@ -1,21 +1,17 @@ package org.infinispan.interceptors.impl; -import static org.infinispan.interceptors.impl.PassivationCacheLoaderInterceptor.activateAfterLoad; - import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; import org.infinispan.commands.FlagAffectedCommand; import org.infinispan.container.entries.InternalCacheEntry; import org.infinispan.context.InvocationContext; -import org.infinispan.eviction.impl.ActivationManager; import org.infinispan.factories.annotations.Inject; import org.infinispan.util.concurrent.CompletionStages; import org.infinispan.util.concurrent.DataOperationOrderer; public class PassivationClusteredCacheLoaderInterceptor extends ClusteredCacheLoaderInterceptor { @Inject DataOperationOrderer orderer; - @Inject ActivationManager activationManager; @Override public CompletionStage> loadAndStoreInDataContainer(InvocationContext ctx, Object key, @@ -29,14 +25,7 @@ public CompletionStage> loadAndStoreInDataContainer(Inv } else { retrievalStage = super.loadAndStoreInDataContainer(ctx, key, segment, cmd); } - if (CompletionStages.isCompletedSuccessfully(retrievalStage)) { - InternalCacheEntry ice = CompletionStages.join(retrievalStage); - activateAfterLoad(key, segment, orderer, activationManager, future, ice, null); - return retrievalStage; - } else { - return retrievalStage.whenComplete((value, t) -> { - activateAfterLoad(key, segment, orderer, activationManager, future, value, t); - }); - } + + return retrievalStage.whenComplete((v, t) -> orderer.completeOperation(key, future, DataOperationOrderer.Operation.READ)); } } diff --git a/core/src/main/java/org/infinispan/interceptors/impl/PassivationWriterInterceptor.java b/core/src/main/java/org/infinispan/interceptors/impl/PassivationWriterInterceptor.java index 3247aca001f5..42f7c0ab63e7 100644 --- a/core/src/main/java/org/infinispan/interceptors/impl/PassivationWriterInterceptor.java +++ b/core/src/main/java/org/infinispan/interceptors/impl/PassivationWriterInterceptor.java @@ -1,48 +1,44 @@ package org.infinispan.interceptors.impl; -import static org.infinispan.persistence.manager.PersistenceManager.AccessMode.BOTH; -import static org.infinispan.persistence.manager.PersistenceManager.AccessMode.PRIVATE; +import java.util.concurrent.CompletionStage; import org.infinispan.commands.FlagAffectedCommand; -import org.infinispan.commands.write.ClearCommand; +import org.infinispan.commands.write.PutMapCommand; +import org.infinispan.commands.write.RemoveCommand; +import org.infinispan.commons.util.concurrent.CompletableFutures; import org.infinispan.context.InvocationContext; -import org.infinispan.context.impl.FlagBitSets; -import org.infinispan.factories.annotations.Inject; -import org.infinispan.interceptors.DDAsyncInterceptor; -import org.infinispan.persistence.manager.PersistenceManager; -import org.infinispan.util.logging.Log; -import org.infinispan.util.logging.LogFactory; +import org.infinispan.context.impl.TxInvocationContext; +import org.infinispan.interceptors.InvocationStage; +import org.infinispan.transaction.impl.AbstractCacheTransaction; /** - * Handles store write operations when passivation enabled that don't entail reading the entry first + * Passivation writer ignores any create/modify operations and only does removals. The writes are done via eviction + * or shutdown only. * * @author William Burns - * @since 9.0 + * @since 15.0 */ -public class PassivationWriterInterceptor extends DDAsyncInterceptor { - @Inject protected PersistenceManager persistenceManager; +public class PassivationWriterInterceptor extends CacheWriterInterceptor { - private static final Log log = LogFactory.getLog(PassivationWriterInterceptor.class); - - protected Log getLog() { - return log; + @Override + CompletionStage storeEntry(InvocationContext ctx, Object key, FlagAffectedCommand command, boolean incrementStats) { + return CompletableFutures.completedNull(); } @Override - public Object visitClearCommand(InvocationContext ctx, ClearCommand command) throws Throwable { - if (isStoreEnabled(command) && !ctx.isInTxScope()) - return asyncInvokeNext(ctx, command, persistenceManager.clearAllStores(ctx.isOriginLocal() ? BOTH : PRIVATE)); + protected Object handlePutMapCommandReturn(InvocationContext rCtx, PutMapCommand putMapCommand, Object rv) { + return rv; + } - return invokeNext(ctx, command); + @Override + protected InvocationStage store(TxInvocationContext ctx) throws Throwable { + CompletionStage batchStage = persistenceManager.performBatch(ctx, ((writeCommand, k, v) -> + isProperWriter(ctx, writeCommand, k) && v.isRemoved())); + return asyncValue(batchStage); } - protected boolean isStoreEnabled(FlagAffectedCommand command) { - if (command.hasAnyFlag(FlagBitSets.SKIP_CACHE_STORE)) { - if (log.isTraceEnabled()) { - log.trace("Skipping cache store since the call contain a skip cache store flag"); - } - return false; - } - return true; + @Override + boolean shouldReplicateRemove(InvocationContext ctx, RemoveCommand removeCommand) { + return removeCommand.isSuccessful(); } } diff --git a/core/src/main/java/org/infinispan/interceptors/impl/TransactionalStoreInterceptor.java b/core/src/main/java/org/infinispan/interceptors/impl/TransactionalStoreInterceptor.java index 8bd8852d357a..42679d0248f7 100644 --- a/core/src/main/java/org/infinispan/interceptors/impl/TransactionalStoreInterceptor.java +++ b/core/src/main/java/org/infinispan/interceptors/impl/TransactionalStoreInterceptor.java @@ -36,7 +36,7 @@ public Object visitPrepareCommand(TxInvocationContext ctx, PrepareCommand comman // Persist the modifications in one phase // After they were successfully applied in the data container - return asyncValue(persistenceManager.performBatch(ctx, (writeCommand, o) -> true)); + return asyncValue(persistenceManager.performBatch(ctx, (writeCommand, o, e) -> true)); }); } } diff --git a/core/src/main/java/org/infinispan/interceptors/locking/ClusteringDependentLogic.java b/core/src/main/java/org/infinispan/interceptors/locking/ClusteringDependentLogic.java index 4ba60c980ae6..c9ec5dad8282 100644 --- a/core/src/main/java/org/infinispan/interceptors/locking/ClusteringDependentLogic.java +++ b/core/src/main/java/org/infinispan/interceptors/locking/ClusteringDependentLogic.java @@ -168,8 +168,6 @@ abstract class AbstractClusteringDependentLogic implements ClusteringDependentLo private WriteSkewHelper.KeySpecificLogic keySpecificLogic; private EntryLoader entryLoader; private volatile boolean writeOrdering = false; - private volatile boolean passivation = false; - @Start public void start() { updateOrdering(configuration.persistence().usingStores()); @@ -211,8 +209,6 @@ private void updateOrdering(boolean usingStores) { // Stores require writing entries to data container after loading - in an atomic fashion if (memoryConfiguration.isEvictionEnabled() || usingStores) { writeOrdering = true; - // Passivation also has some additional things required when doing writes - passivation = usingStores && persistenceConfiguration.passivation(); } } @@ -242,9 +238,6 @@ public final CompletionStage commitEntry(CacheEntry entry, FlagAffectedCom if (!writeOrdering) { return commitSingleEntry(entry, command, ctx, trackFlag, l1Invalidation); } - if (passivation) { - return commitEntryPassivation(entry, command, ctx, trackFlag, l1Invalidation); - } return commitEntryOrdered(entry, command, ctx, trackFlag, l1Invalidation); } @@ -267,59 +260,10 @@ private CompletionStage commitClearCommand(DataContainer d } } - private CompletionStage commitEntryPassivation(CacheEntry entry, FlagAffectedCommand command, InvocationContext ctx, - Flag trackFlag, boolean l1Invalidation) { - // To clarify the below section these operations must be done in order and cannot be be reordered otherwise - // it can cause data guarantee issues with other operations - // 1. Acquire the order guarantee via orderer.orderOn - // 2. Query the data container if the entry is in memory - // 3. Update the in memory contents - // 4. Remove the entry from the store if the entry was not in memory - // 5. Complete/release the order guarantee - - Object key = entry.getKey(); - int segment = SegmentSpecificCommand.extractSegment(command, key, keyPartioner); - CompletableFuture ourFuture = new CompletableFuture<>(); - // If this future is not null it means there is another pending read/write/eviction for this key, thus - // we have to wait on it before performing our commit to ensure data is updated properly - CompletionStage waitingFuture = orderer.orderOn(key, ourFuture); - // We don't want to waste time removing an entry from the store if it is in the data container - // We use peek here instead of containsKey as the value could be expired - if so we want to make sure - // passivation manager knows the key is not in the store - - CompletionStage chainedStage; - if (waitingFuture != null) { - // We have to wait on another operation to complete before doing the update - chainedStage = waitingFuture.thenCompose(ignore -> activateKey(key, segment, entry, command, ctx, trackFlag, l1Invalidation)); - } else { - chainedStage = activateKey(key, segment, entry, command, ctx, trackFlag, l1Invalidation); - } - // After everything is done we have to make sure to complete our future - return chainedStage.whenComplete((ignore, ignoreT) -> orderer.completeOperation(key, ourFuture, operation(entry))); - } - private static DataOperationOrderer.Operation operation(CacheEntry entry) { return entry.isRemoved() ? DataOperationOrderer.Operation.REMOVE : DataOperationOrderer.Operation.WRITE; } - private CompletionStage activateKey(Object key, int segment, CacheEntry entry, FlagAffectedCommand command, - InvocationContext ctx, Flag trackFlag, boolean l1Invalidation) { - // If entry wasn't in container we should activate to remove from store - boolean shouldActivate = dataContainer.peek(segment, key) == null; - CompletionStage commitStage = commitSingleEntry(entry, command, ctx, trackFlag, l1Invalidation); - if (shouldActivate) { - return commitStage.thenCompose(ignore1 -> { - if (log.isTraceEnabled()) { - log.tracef("Activating entry for key %s due to update in dataContainer", key); - } - return activationManager.activateAsync(key, segment); - }); - } else if (log.isTraceEnabled()) { - log.tracef("Skipping removal from store as %s was in the data container", key); - } - return commitStage; - } - private CompletionStage commitEntryOrdered(CacheEntry entry, FlagAffectedCommand command, InvocationContext ctx, Flag trackFlag, boolean l1Invalidation) { Object key = entry.getKey(); diff --git a/core/src/main/java/org/infinispan/interceptors/xsite/NonTransactionalBackupInterceptor.java b/core/src/main/java/org/infinispan/interceptors/xsite/NonTransactionalBackupInterceptor.java index 0f04e7378192..32f255b3a306 100644 --- a/core/src/main/java/org/infinispan/interceptors/xsite/NonTransactionalBackupInterceptor.java +++ b/core/src/main/java/org/infinispan/interceptors/xsite/NonTransactionalBackupInterceptor.java @@ -140,6 +140,12 @@ private Object handleMultipleKeysWriteReturn(InvocationContext ctx, WriteCommand } return rv; } + if (!writeCommand.shouldReplicate(ctx, true)) { + if (log.isTraceEnabled()) { + log.tracef("Command %s says not to replicate", writeCommand); + } + return rv; + } Map map = new HashMap<>(); // must support null values LocalizedCacheTopology localizedCacheTopology = clusteringDependentLogic.getCacheTopology(); for (Object key : writeCommand.getAffectedKeys()) { diff --git a/core/src/main/java/org/infinispan/persistence/manager/PersistenceManager.java b/core/src/main/java/org/infinispan/persistence/manager/PersistenceManager.java index b0e1ab821750..f53da78b0b91 100644 --- a/core/src/main/java/org/infinispan/persistence/manager/PersistenceManager.java +++ b/core/src/main/java/org/infinispan/persistence/manager/PersistenceManager.java @@ -10,14 +10,16 @@ import org.infinispan.commands.write.WriteCommand; import org.infinispan.commons.api.Lifecycle; import org.infinispan.commons.util.IntSet; +import org.infinispan.commons.util.concurrent.CompletableFutures; import org.infinispan.configuration.cache.StoreConfiguration; +import org.infinispan.container.entries.MVCCEntry; import org.infinispan.context.InvocationContext; import org.infinispan.context.impl.TxInvocationContext; import org.infinispan.persistence.spi.AdvancedCacheLoader; import org.infinispan.persistence.spi.MarshallableEntry; import org.infinispan.persistence.spi.PersistenceException; import org.infinispan.transaction.impl.AbstractCacheTransaction; -import org.infinispan.commons.util.concurrent.CompletableFutures; +import org.infinispan.util.function.TriPredicate; import org.reactivestreams.Publisher; import io.reactivex.rxjava3.core.Flowable; @@ -350,11 +352,11 @@ CompletionStage writeMapCommand(PutMapCommand putMapCommand, InvocationCon /** * Writes a batch for the given modifications in the transactional context * @param invocationContext transactional context - * @param commandKeyPredicate predicate to control if a key/command combination should be accepted + * @param commandKeyPredicate predicate to control if a key/value/command combination should be accepted * @return a stage of how many writes were performed */ CompletionStage performBatch(TxInvocationContext invocationContext, - BiPredicate commandKeyPredicate); + TriPredicate> commandKeyPredicate); /** * Writes the entries to the stores that pass the given predicate diff --git a/core/src/main/java/org/infinispan/persistence/manager/PersistenceManagerImpl.java b/core/src/main/java/org/infinispan/persistence/manager/PersistenceManagerImpl.java index 52448e313925..469649aa5008 100644 --- a/core/src/main/java/org/infinispan/persistence/manager/PersistenceManagerImpl.java +++ b/core/src/main/java/org/infinispan/persistence/manager/PersistenceManagerImpl.java @@ -84,6 +84,7 @@ import org.infinispan.util.concurrent.BlockingManager; import org.infinispan.util.concurrent.CompletionStages; import org.infinispan.util.concurrent.NonBlockingManager; +import org.infinispan.util.function.TriPredicate; import org.infinispan.util.logging.Log; import org.infinispan.util.logging.LogFactory; import org.reactivestreams.Publisher; @@ -1176,13 +1177,14 @@ public CompletionStage writeEntries(Iterable writeMapCommand(PutMapCommand putMapCommand, InvocationContext ctx, BiPredicate commandKeyPredicate) { - Flowable> mvccEntryFlowable = entriesFromCommand(putMapCommand, ctx, commandKeyPredicate); + Flowable> mvccEntryFlowable = entriesFromCommand(putMapCommand, ctx, (c, k, e) -> + commandKeyPredicate.test(c, k)); return batchOperation(mvccEntryFlowable, ctx, NonBlockingStore::batch); } @Override public CompletionStage performBatch(TxInvocationContext ctx, - BiPredicate commandKeyPredicate) { + TriPredicate> commandKeyPredicate) { Flowable> mvccEntryFlowable = toMvccEntryFlowable(ctx, commandKeyPredicate); return batchOperation(mvccEntryFlowable, ctx, NonBlockingStore::batch); } @@ -1364,14 +1366,14 @@ private Flowable> filterSharedSegment * @return a Flowable containing MVCCEntry(s) for the modifications in the tx context */ private Flowable> toMvccEntryFlowable(TxInvocationContext ctx, - BiPredicate commandKeyPredicate) { + TriPredicate> commandKeyPredicate) { return Flowable.fromIterable(ctx.getCacheTransaction().getAllModifications()) .filter(writeCommand -> !writeCommand.hasAnyFlag(FlagBitSets.SKIP_CACHE_STORE | FlagBitSets.ROLLING_UPGRADE)) .concatMap(writeCommand -> entriesFromCommand(writeCommand, ctx, commandKeyPredicate)); } private Flowable> entriesFromCommand(WCT writeCommand, InvocationContext ctx, - BiPredicate commandKeyPredicate) { + TriPredicate> commandKeyPredicate) { if (writeCommand instanceof DataWriteCommand) { Object key = ((DataWriteCommand) writeCommand).getKey(); MVCCEntry entry = acquireKeyFromContext(ctx, writeCommand, key, commandKeyPredicate); @@ -1391,15 +1393,13 @@ private Flowable> entriesFromCo } private MVCCEntry acquireKeyFromContext(InvocationContext ctx, WCT command, Object key, - BiPredicate commandKeyPredicate) { - if (commandKeyPredicate == null || commandKeyPredicate.test(command, key)) { - //noinspection unchecked - MVCCEntry entry = (MVCCEntry) ctx.lookupEntry(key); - if (entry.isChanged()) { - return entry; - } + TriPredicate> commandKeyPredicate) { + //noinspection unchecked + MVCCEntry entry = (MVCCEntry) ctx.lookupEntry(key); + if (commandKeyPredicate != null && !commandKeyPredicate.test(command, key, entry) || !entry.isChanged()) { + return null; } - return null; + return entry; } /** diff --git a/core/src/main/java/org/infinispan/persistence/sifs/Compactor.java b/core/src/main/java/org/infinispan/persistence/sifs/Compactor.java index 8c06e4a3c367..7952dc433ced 100644 --- a/core/src/main/java/org/infinispan/persistence/sifs/Compactor.java +++ b/core/src/main/java/org/infinispan/persistence/sifs/Compactor.java @@ -120,8 +120,16 @@ ConcurrentMap getFileStats() { return fileStats; } - boolean addFreeFile(int file, int expectedSize, int freeSize, long expirationTime) { - return addFreeFile(file, expectedSize, freeSize, expirationTime, true); + void addLogFileOnShutdown(int file, long expirationTime) { + int fileSize = (int) fileProvider.getFileSize(file); + // Note this doesn't have to be concurrent as this method is only invoked on shutdown when no additional + // files will be requested concurrently + Stats stats = new Stats(fileSize, 0, expirationTime); + Stats prevStats = fileStats.put(file, stats); + if (prevStats != null) { + stats.free.addAndGet(prevStats.getFree()); + } + log.tracef("Added log file %s to compactor at shutdown with total size %s and free size %s", file, fileSize, stats.getFree()); } boolean addFreeFile(int file, int expectedSize, int freeSize, long expirationTime, boolean canScheduleCompaction) { @@ -570,7 +578,9 @@ private void compactSingleFile(CompactionRequest compactionRequest, log.tracef("No index found for key %s, but it is a logFile, ignoring rest of the file", key); break; } - throw new NullPointerException("No index info found for key: " + key + " when processing file " + scheduledFile); + log.tracef("No index found for key %s, dropping - assuming lost due to segments removed", key); + scheduledOffset += header.totalLength(); + continue; } if (info.numRecords <= 0) { throw new IllegalArgumentException("Number of records " + info.numRecords + " for index of key " + key + " should be more than zero!"); @@ -583,6 +593,11 @@ private void compactSingleFile(CompactionRequest compactionRequest, // We can only truncate expired entries if this was compacted with purge expire if (expiredIndex != null) { EntryRecord record = index.getRecordEvenIfExpired(key, segment, serializedKey); + if (record == null) { + log.tracef("Key %s is not in index to do expiration event - assuming lost due to segments removed", key); + scheduledOffset += header.totalLength(); + continue; + } truncate = true; expiredIndex.add(record); // If there are more entries we cannot drop the index as we need a tombstone @@ -669,8 +684,9 @@ private void compactSingleFile(CompactionRequest compactionRequest, try { EntryInfo info = index.getInfo(key, segment, serializedKey); if (info == null) { - throw new IllegalStateException(String.format( - "%s was not found in index but it was not in temporary table and there's entry on %d:%d", key, scheduledFile, indexedOffset)); + log.tracef("Key %s was not found in index or temporary table assuming it is gone from removing segments, dropping", key); + scheduledOffset += header.totalLength(); + continue; } else { update = info.file == scheduledFile && info.offset == indexedOffset; } diff --git a/core/src/main/java/org/infinispan/persistence/sifs/EntryInfo.java b/core/src/main/java/org/infinispan/persistence/sifs/EntryInfo.java index e67bee946b83..073095217087 100644 --- a/core/src/main/java/org/infinispan/persistence/sifs/EntryInfo.java +++ b/core/src/main/java/org/infinispan/persistence/sifs/EntryInfo.java @@ -5,15 +5,13 @@ */ public class EntryInfo extends EntryPosition { public final int numRecords; - public final int cacheSegment; - public EntryInfo(int file, int offset, int numRecords, int cacheSegment) { + public EntryInfo(int file, int offset, int numRecords) { super(file, offset); this.numRecords = numRecords; - this.cacheSegment = cacheSegment; } public String toString() { - return String.format("[%d:%d] containing %d records in segment %d", file, offset, numRecords, cacheSegment); + return String.format("[%d:%d] containing %d records", file, offset, numRecords); } } diff --git a/core/src/main/java/org/infinispan/persistence/sifs/FileProvider.java b/core/src/main/java/org/infinispan/persistence/sifs/FileProvider.java index 37d0e31ac42e..eb0c73d8ed20 100644 --- a/core/src/main/java/org/infinispan/persistence/sifs/FileProvider.java +++ b/core/src/main/java/org/infinispan/persistence/sifs/FileProvider.java @@ -40,7 +40,7 @@ public class FileProvider { private static final String REGEX_FORMAT = "^%s[0-9]+$"; private static final boolean ATTEMPT_PMEM; - private final File dataDir; + private final File directoryFile; private final int openFileLimit; private final ArrayBlockingQueue recordQueue; private final ConcurrentMap openFiles = new ConcurrentHashMap<>(); @@ -51,6 +51,8 @@ public class FileProvider { private final String prefix; private final int maxFileSize; + private boolean canTryPmem = true; + private int nextFileId = 0; static { @@ -65,14 +67,16 @@ public class FileProvider { ATTEMPT_PMEM = attemptPmem; } - public FileProvider(Path dataDir, int openFileLimit, String prefix, int maxFileSize) { + public FileProvider(Path fileDirectory, int openFileLimit, String prefix, int maxFileSize) { this.openFileLimit = openFileLimit; this.recordQueue = new ArrayBlockingQueue<>(openFileLimit); - this.dataDir = dataDir.toFile(); + this.directoryFile = fileDirectory.toFile(); this.prefix = prefix; this.maxFileSize = maxFileSize; - if (!SecurityActions.createDirectoryIfNeeded(this.dataDir)) { - throw PERSISTENCE.directoryCannotBeCreated(this.dataDir.getAbsolutePath()); + try { + Files.createDirectories(fileDirectory); + } catch (IOException e) { + throw PERSISTENCE.directoryCannotBeCreated(this.directoryFile.getAbsolutePath()); } } @@ -85,6 +89,23 @@ public boolean isLogFile(int fileId) { } } + public Handle getFileIfOpen(int fileId) { + lock.readLock().lock(); + try { + Record record = openFiles.get(fileId); + if (record != null) { + synchronized (record) { + if (record.isOpen()) { + return new Handle(record); + } + } + } + return null; + } finally { + lock.readLock().unlock(); + } + } + public Handle getFile(int fileId) throws IOException { lock.readLock().lock(); try { @@ -101,6 +122,7 @@ public Handle getFile(int fileId) throws IOException { break; } } + Thread.yield(); } // now we have either removed some other opened file or incremented the value below limit for (;;) { @@ -170,7 +192,7 @@ private String fileIdToString(int fileId) { // Package private for tests File newFile(int fileId) { - return new File(dataDir, fileIdToString(fileId)); + return new File(directoryFile, fileIdToString(fileId)); } private boolean tryCloseFile() throws IOException { @@ -207,9 +229,10 @@ protected FileChannel openChannel(int fileId) throws FileNotFoundException { protected FileChannel openChannel(File file, boolean create, boolean readSharedMeadata) throws FileNotFoundException { log.debugf("openChannel(%s)", file.getAbsolutePath()); - FileChannel fileChannel = ATTEMPT_PMEM ? PmemUtilWrapper.pmemChannelFor(file, maxFileSize, create, readSharedMeadata) : null; + FileChannel fileChannel = ATTEMPT_PMEM && canTryPmem ? PmemUtilWrapper.pmemChannelFor(file, maxFileSize, create, readSharedMeadata) : null; if (fileChannel == null) { + canTryPmem = false; if (create) { fileChannel = SecurityActions.createChannel(file); } else { @@ -255,7 +278,7 @@ public CloseableIterator getFileIterator() { lock.readLock().lock(); try { Set set = new HashSet<>(); - for (String file : dataDir.list()) { + for (String file : directoryFile.list()) { if (file.matches(regex)) { set.add(Integer.parseInt(file.substring(prefix.length()))); } @@ -272,7 +295,7 @@ public boolean hasFiles() { String regex = String.format(REGEX_FORMAT, prefix); lock.readLock().lock(); try { - for (String file : dataDir.list()) { + for (String file : directoryFile.list()) { if (file.matches(regex)) { return true; } @@ -296,7 +319,7 @@ public void clear() throws IOException { } if (!recordQueue.isEmpty()) throw new IllegalStateException(); if (!openFiles.isEmpty()) throw new IllegalStateException(); - File[] files = dataDir.listFiles(); + File[] files = directoryFile.listFiles(); if (files != null) { for (File file : files) { Files.delete(file.toPath()); @@ -409,6 +432,21 @@ public long getFileSize() throws IOException { public int getFileId() { return record.getFileId(); } + + public void truncate(long i) throws IOException { + if (!usable) throw new IllegalStateException(); + record.getFileChannel().truncate(i); + } + + public int write(ByteBuffer buffer, long l) throws IOException { + if (!usable) throw new IllegalStateException(); + return record.getFileChannel().write(buffer, l); + } + + public void force(boolean metaData) throws IOException { + if (!usable) throw new IllegalStateException(); + record.getFileChannel().force(metaData); + } } private class Record { diff --git a/core/src/main/java/org/infinispan/persistence/sifs/Index.java b/core/src/main/java/org/infinispan/persistence/sifs/Index.java index 7061933bbec8..b3ddb0799e24 100644 --- a/core/src/main/java/org/infinispan/persistence/sifs/Index.java +++ b/core/src/main/java/org/infinispan/persistence/sifs/Index.java @@ -26,7 +26,9 @@ import org.infinispan.commons.io.UnsignedNumeric; import org.infinispan.commons.time.TimeService; import org.infinispan.commons.util.IntSet; +import org.infinispan.commons.util.IntSets; import org.infinispan.commons.util.concurrent.CompletableFutures; +import org.infinispan.executors.LimitedExecutor; import org.infinispan.util.concurrent.AggregateCompletionStage; import org.infinispan.util.concurrent.CompletionStages; import org.infinispan.util.concurrent.NonBlockingManager; @@ -38,6 +40,7 @@ import io.reactivex.rxjava3.processors.FlowableProcessor; import io.reactivex.rxjava3.processors.UnicastProcessor; import io.reactivex.rxjava3.schedulers.Schedulers; +import net.jcip.annotations.GuardedBy; /** * Keeps the entry positions persisted in a file. It consists of couple of segments, each for one modulo-range of key's @@ -60,18 +63,32 @@ class Index { private static final int INDEX_FILE_HEADER_SIZE = 34; private final NonBlockingManager nonBlockingManager; - private final FileProvider fileProvider; + private final FileProvider dataFileProvider; + private final FileProvider indexFileProvider; private final Path indexDir; private final Compactor compactor; private final int minNodeSize; private final int maxNodeSize; private final ReadWriteLock lock = new ReentrantReadWriteLock(); + @GuardedBy("lock") private final Segment[] segments; + @GuardedBy("lock") + private final FlowableProcessor[] flowableProcessors; private final TimeService timeService; private final File indexSizeFile; public final AtomicLongArray sizePerSegment; - private final FlowableProcessor[] flowableProcessors; + private final TemporaryTable temporaryTable; + + private final Executor executor; + + private final Throwable END_EARLY = new Throwable("SIFS Index stopping"); + // This is used to signal that a segment is not currently being used + private final Segment emptySegment; + private final FlowableProcessor emptyFlowable; + + @GuardedBy("lock") + private CompletionStage removeSegmentsStage = CompletableFutures.completedNull(); private final IndexNode.OverwriteHook movedHook = new IndexNode.OverwriteHook() { @Override @@ -108,49 +125,86 @@ public void setOverwritten(IndexRequest request, int cacheSegment, boolean overw } }; - public Index(NonBlockingManager nonBlockingManager, FileProvider fileProvider, Path indexDir, int segments, - int cacheSegments, int minNodeSize, int maxNodeSize, TemporaryTable temporaryTable, Compactor compactor, - TimeService timeService) throws IOException { + public Index(NonBlockingManager nonBlockingManager, FileProvider dataFileProvider, Path indexDir, int cacheSegments, + int minNodeSize, int maxNodeSize, TemporaryTable temporaryTable, Compactor compactor, + TimeService timeService, Executor executor, int maxOpenFiles) throws IOException { this.nonBlockingManager = nonBlockingManager; - this.fileProvider = fileProvider; + this.dataFileProvider = dataFileProvider; this.compactor = compactor; this.timeService = timeService; this.indexDir = indexDir; this.minNodeSize = minNodeSize; this.maxNodeSize = maxNodeSize; this.sizePerSegment = new AtomicLongArray(cacheSegments); - indexDir.toFile().mkdirs(); + this.indexFileProvider = new FileProvider(indexDir, maxOpenFiles, "index.", Integer.MAX_VALUE); this.indexSizeFile = new File(indexDir.toFile(), "index-count"); - this.segments = new Segment[segments]; - this.flowableProcessors = new FlowableProcessor[segments]; - for (int i = 0; i < segments; ++i) { - UnicastProcessor flowableProcessor = UnicastProcessor.create(); - Segment segment = new Segment(this, i, temporaryTable); + this.segments = new Segment[cacheSegments]; + this.flowableProcessors = new FlowableProcessor[cacheSegments]; + + this.temporaryTable = temporaryTable; + // Limits the amount of concurrent updates we do to the underlying indices to be based on the number of cache + // segments. Note that this uses blocking threads so this number is still limited by that as well + int concurrency = Math.max(cacheSegments >> 4, 1); + this.executor = new LimitedExecutor("sifs-index", executor, concurrency); + this.emptySegment = new Segment(this, -1, temporaryTable); + this.emptySegment.complete(null); + + this.emptyFlowable = UnicastProcessor.create() + .toSerialized(); + emptyFlowable.subscribe(this::handleNonOwnedIndexRequest, + e -> log.fatal("Error encountered with index, SIFS may not operate properly.", e)); + } - this.segments[i] = segment; - // It is possible to write from multiple threads - this.flowableProcessors[i] = flowableProcessor.toSerialized(); + /** + * Method used to handle index requests where we do not own the given segment. In this case we just effectively + * discard the request when possible and update the compactor to have the data as free so it can remove the value + * later. + * @param ir request that was sent after we don't own the segment + */ + private void handleNonOwnedIndexRequest(IndexRequest ir) { + switch (ir.getType()) { + case UPDATE: + case MOVED: + // We no longer own the segment so just treat the data as free for compactor purposes + // Note we leave the existing value alone as the removeSegments will take care of that + compactor.free(ir.getFile(), ir.getSize()); + break; + case FOUND_OLD: + throw new IllegalStateException("This is only possible when building the index"); + case SYNC_REQUEST: + Runnable runnable = (Runnable) ir.getKey(); + runnable.run(); + break; + case CLEAR: + case DROPPED: + // Both drop and clear do nothing as we are already removing the index } + ir.complete(null); } private boolean checkForExistingIndexSizeFile() { - int storeSegments = flowableProcessors.length; int cacheSegments = sizePerSegment.length(); boolean validCount = false; try (RandomAccessFile indexCount = new RandomAccessFile(indexSizeFile, "r")) { - int storeSegmentsCount = UnsignedNumeric.readUnsignedInt(indexCount); int cacheSegmentsCount = UnsignedNumeric.readUnsignedInt(indexCount); - if (storeSegmentsCount == storeSegments && cacheSegmentsCount == cacheSegments) { + if (cacheSegmentsCount == cacheSegments) { for (int i = 0; i < sizePerSegment.length(); ++i) { long value = UnsignedNumeric.readUnsignedLong(indexCount); + if (value < 0) { + log.tracef("Found an invalid size for a segment, assuming index is a different format"); + return false; + } sizePerSegment.set(i, value); } - validCount = true; + if (indexCount.read() != -1) { + log.tracef("Previous index file has more bytes than desired, assuming index is a different format"); + } else { + validCount = true; + } } else { - log.tracef("Previous index file store segments " + storeSegmentsCount + " doesn't match configured" + - " store segments " + storeSegments + " or index file cache segments " + cacheSegmentsCount + " doesn't match configured" + + log.tracef("Previous index file cache segments " + cacheSegmentsCount + " doesn't match configured" + " cache segments " + cacheSegments); } } catch (IOException e) { @@ -162,19 +216,16 @@ private boolean checkForExistingIndexSizeFile() { return validCount; } - public static byte[] toIndexKey(int cacheSegment, org.infinispan.commons.io.ByteBuffer buffer) { - return toIndexKey(cacheSegment, buffer.getBuf(), buffer.getOffset(), buffer.getLength()); - } - - static byte[] toIndexKey(int cacheSegment, byte[] bytes) { - return toIndexKey(cacheSegment, bytes, 0, bytes.length); + public static byte[] toIndexKey(org.infinispan.commons.io.ByteBuffer buffer) { + return toIndexKey(buffer.getBuf(), buffer.getOffset(), buffer.getLength()); } - static byte[] toIndexKey(int cacheSegment, byte[] bytes, int offset, int length) { - int segmentBytes = UnsignedNumeric.sizeUnsignedInt(cacheSegment); - byte[] indexKey = new byte[length + segmentBytes]; - UnsignedNumeric.writeUnsignedInt(indexKey, 0, cacheSegment); - System.arraycopy(bytes, 0, indexKey, segmentBytes + offset, length); + static byte[] toIndexKey(byte[] bytes, int offset, int length) { + if (offset == 0 && length == bytes.length) { + return bytes; + } + byte[] indexKey = new byte[length]; + System.arraycopy(bytes, 0, indexKey, 0, length); return indexKey; } @@ -233,21 +284,20 @@ public void reset() throws IOException { * Get record or null if expired */ public EntryRecord getRecord(Object key, int cacheSegment, org.infinispan.commons.io.ByteBuffer serializedKey) throws IOException { - return getRecord(key, cacheSegment, toIndexKey(cacheSegment, serializedKey), IndexNode.ReadOperation.GET_RECORD); + return getRecord(key, cacheSegment, toIndexKey(serializedKey), IndexNode.ReadOperation.GET_RECORD); } /** * Get record (even if expired) or null if not present */ public EntryRecord getRecordEvenIfExpired(Object key, int cacheSegment, byte[] serializedKey) throws IOException { - return getRecord(key, cacheSegment, toIndexKey(cacheSegment, serializedKey), IndexNode.ReadOperation.GET_EXPIRED_RECORD); + return getRecord(key, cacheSegment, serializedKey, IndexNode.ReadOperation.GET_EXPIRED_RECORD); } private EntryRecord getRecord(Object key, int cacheSegment, byte[] indexKey, IndexNode.ReadOperation readOperation) throws IOException { - int segment = (key.hashCode() & Integer.MAX_VALUE) % segments.length; lock.readLock().lock(); try { - return IndexNode.applyOnLeaf(segments[segment], cacheSegment, indexKey, segments[segment].rootReadLock(), readOperation); + return IndexNode.applyOnLeaf(segments[cacheSegment], cacheSegment, indexKey, segments[cacheSegment].rootReadLock(), readOperation); } finally { lock.readLock().unlock(); } @@ -257,10 +307,9 @@ private EntryRecord getRecord(Object key, int cacheSegment, byte[] indexKey, Ind * Get position or null if expired */ public EntryPosition getPosition(Object key, int cacheSegment, org.infinispan.commons.io.ByteBuffer serializedKey) throws IOException { - int segment = (key.hashCode() & Integer.MAX_VALUE) % segments.length; lock.readLock().lock(); try { - return IndexNode.applyOnLeaf(segments[segment], cacheSegment, toIndexKey(cacheSegment, serializedKey), segments[segment].rootReadLock(), IndexNode.ReadOperation.GET_POSITION); + return IndexNode.applyOnLeaf(segments[cacheSegment], cacheSegment, toIndexKey(serializedKey), segments[cacheSegment].rootReadLock(), IndexNode.ReadOperation.GET_POSITION); } finally { lock.readLock().unlock(); } @@ -270,10 +319,9 @@ public EntryPosition getPosition(Object key, int cacheSegment, org.infinispan.co * Get position + numRecords, without expiration */ public EntryInfo getInfo(Object key, int cacheSegment, byte[] serializedKey) throws IOException { - int segment = (key.hashCode() & Integer.MAX_VALUE) % segments.length; lock.readLock().lock(); try { - return IndexNode.applyOnLeaf(segments[segment], cacheSegment, toIndexKey(cacheSegment, serializedKey), segments[segment].rootReadLock(), IndexNode.ReadOperation.GET_INFO); + return IndexNode.applyOnLeaf(segments[cacheSegment], cacheSegment, serializedKey, segments[cacheSegment].rootReadLock(), IndexNode.ReadOperation.GET_INFO); } finally { lock.readLock().unlock(); } @@ -285,6 +333,10 @@ public CompletionStage clear() { try { AggregateCompletionStage stage = CompletionStages.aggregateCompletionStage(); for (FlowableProcessor processor : flowableProcessors) { + // Ignore emptyFlowable as we use this to signal that we don't own that segment anymore + if (processor == emptyFlowable) { + continue; + } IndexRequest clearRequest = IndexRequest.clearRequest(); processor.onNext(clearRequest); stage.dependsOn(clearRequest); @@ -299,8 +351,7 @@ public CompletionStage clear() { } public CompletionStage handleRequest(IndexRequest indexRequest) { - int processor = (indexRequest.getKey().hashCode() & Integer.MAX_VALUE) % segments.length; - flowableProcessors[processor].onNext(indexRequest); + flowableProcessors[indexRequest.getSegment()].onNext(indexRequest); return indexRequest; } @@ -320,28 +371,36 @@ public void deleteFileAsync(int fileId) { ensureRunOnLast(() -> { // After all indexes have ensured they have processed all requests - the last one will delete the file // This guarantees that the index can't see an outdated value - fileProvider.deleteFile(fileId); + dataFileProvider.deleteFile(fileId); compactor.releaseStats(fileId); }); } public CompletionStage stop() throws InterruptedException { - for (FlowableProcessor flowableProcessor : flowableProcessors) { - flowableProcessor.onComplete(); - } + AggregateCompletionStage aggregateCompletionStage; + lock.readLock().lock(); + try { + for (FlowableProcessor flowableProcessor : flowableProcessors) { + flowableProcessor.onComplete(); + } - AggregateCompletionStage aggregateCompletionStage = CompletionStages.aggregateCompletionStage(); - for (Segment segment : segments) { - aggregateCompletionStage.dependsOn(segment); + aggregateCompletionStage = CompletionStages.aggregateCompletionStage(); + for (Segment segment : segments) { + aggregateCompletionStage.dependsOn(segment); + } + + aggregateCompletionStage.dependsOn(removeSegmentsStage); + } finally { + lock.readLock().unlock(); } // After all SIFS segments are complete we write the size return aggregateCompletionStage.freeze().thenRun(() -> { + indexFileProvider.stop(); try { // Create the file first as it should not be present as we deleted during startup indexSizeFile.createNewFile(); try (FileOutputStream indexCountStream = new FileOutputStream(indexSizeFile)) { - UnsignedNumeric.writeUnsignedInt(indexCountStream, segments.length); UnsignedNumeric.writeUnsignedInt(indexCountStream, this.sizePerSegment.length()); for (int i = 0; i < sizePerSegment.length(); ++i) { UnsignedNumeric.writeUnsignedLong(indexCountStream, sizePerSegment.get(i)); @@ -359,7 +418,7 @@ public CompletionStage stop() throws InterruptedException { int file = entry.getKey(); int total = entry.getValue().getTotal(); if (total == -1) { - total = (int) fileProvider.getFileSize(file); + total = (int) dataFileProvider.getFileSize(file); } int free = entry.getValue().getFree(); buffer.putInt(file); @@ -403,19 +462,25 @@ public long getMaxSeqId() throws IOException { return maxSeqId; } - public void start(Executor executor) { - for (int i = 0; i < segments.length; ++i) { - Segment segment = segments[i]; - flowableProcessors[i] - .observeOn(Schedulers.from(executor)) - .subscribe(segment, t -> { - log.error("Error encountered with index, SIFS may not operate properly.", t); - segment.completeExceptionally(t); - }, segment); - } + public void start() { + addSegments(IntSets.immutableRangeSet(segments.length)); + } + + static boolean read(FileProvider.Handle handle, ByteBuffer buffer, long offset) throws IOException { + assert buffer.hasRemaining(); + int read = 0; + do { + int newRead = handle.read(buffer, offset + read); + if (newRead < 0) { + return false; + } + read += newRead; + } while (buffer.hasRemaining()); + return true; } static boolean read(FileChannel channel, ByteBuffer buffer) throws IOException { + assert buffer.hasRemaining(); do { int read = channel.read(buffer); if (read < 0) { @@ -425,7 +490,17 @@ static boolean read(FileChannel channel, ByteBuffer buffer) throws IOException { return true; } + private static long write(FileProvider.Handle handle, ByteBuffer buffer, long offset) throws IOException { + assert buffer.hasRemaining(); + long write = 0; + while (buffer.hasRemaining()) { + write += handle.write(buffer, offset + write); + } + return write; + } + private static void write(FileChannel indexFile, ByteBuffer buffer) throws IOException { + assert buffer.hasRemaining(); do { int written = indexFile.write(buffer); if (written < 0) { @@ -434,69 +509,223 @@ private static void write(FileChannel indexFile, ByteBuffer buffer) throws IOExc } while (buffer.position() < buffer.limit()); } + public CompletionStage addSegments(IntSet addedSegments) { + // Since actualAddSegments doesn't block we try a quick write lock acquisition to possibly avoid context change + if (lock.writeLock().tryLock()) { + try { + actualAddSegments(addedSegments); + } finally { + lock.writeLock().unlock(); + } + return CompletableFutures.completedNull(); + } + return CompletableFuture.runAsync(() -> { + lock.writeLock().lock(); + try { + actualAddSegments(addedSegments); + } finally { + lock.writeLock().unlock(); + } + }, executor); + } + + private void traceSegmentsAdded(IntSet addedSegments) { + IntSet actualAddedSegments = IntSets.mutableEmptySet(segments.length); + for (PrimitiveIterator.OfInt segmentIter = addedSegments.iterator(); segmentIter.hasNext(); ) { + int i = segmentIter.nextInt(); + if (segments[i] == null || segments[i] == emptySegment) { + actualAddedSegments.add(i); + } + } + log.tracef("Adding segments %s to SIFS index", actualAddedSegments); + } + + private void actualAddSegments(IntSet addedSegments) { + if (log.isTraceEnabled()) { + traceSegmentsAdded(addedSegments); + } + for (PrimitiveIterator.OfInt segmentIter = addedSegments.iterator(); segmentIter.hasNext(); ) { + int i = segmentIter.nextInt(); + + // Segment is already running, don't do anything + if (segments[i] != null && segments[i] != emptySegment) { + continue; + } + + UnicastProcessor flowableProcessor = UnicastProcessor.create(false); + Segment segment = new Segment(this, i, temporaryTable); + + // Note we do not load the segments here as we only have to load them on startup and not if segments are + // added at runtime + this.segments[i] = segment; + // It is possible to write from multiple threads + this.flowableProcessors[i] = flowableProcessor.toSerialized(); + + flowableProcessors[i] + .observeOn(Schedulers.from(executor)) + .subscribe(segment, t -> { + if (t != END_EARLY) + log.error("Error encountered with index, SIFS may not operate properly.", t); + segment.completeExceptionally(t); + }, segment); + } + } + public CompletionStage removeSegments(IntSet removedCacheSegments) { + // Use a try lock to avoid context switch if possible + if (lock.writeLock().tryLock()) { + try { + // This method doesn't block if we can acquire lock immediately, just replaces segments and flowables + // and submits an async task + actualRemoveSegments(removedCacheSegments); + } finally { + lock.writeLock().unlock(); + } + return CompletableFutures.completedNull(); + } + return CompletableFuture.runAsync(() -> { + lock.writeLock().lock(); + try { + actualRemoveSegments(removedCacheSegments); + } finally { + lock.writeLock().unlock(); + } + }, executor); + } + + private void actualRemoveSegments(IntSet removedCacheSegments) { + log.tracef("Removing segments %s from index", removedCacheSegments); + int addedCount = removedCacheSegments.size(); + List removedSegments = new ArrayList<>(addedCount); + List> removedFlowables = new ArrayList<>(addedCount); + CompletableFuture stageWhenComplete = new CompletableFuture<>(); + + for (PrimitiveIterator.OfInt iter = removedCacheSegments.iterator(); iter.hasNext(); ) { + int i = iter.nextInt(); + removedSegments.add(segments[i]); + segments[i] = emptySegment; + removedFlowables.add(flowableProcessors[i]); + flowableProcessors[i] = emptyFlowable; + + sizePerSegment.set(i, 0); + } + // Technically this is an issue with sequential removeSegments being called and then shutdown, but + // we don't support data consistency with non-shared stores in a cluster (this method is only called in a cluster). + removeSegmentsStage = stageWhenComplete; + + executor.execute(() -> { + try { + log.tracef("Cleaning old index information for segments: %s", removedCacheSegments); + // We would love to do this outside of this stage asynchronously but unfortunately we can't say we have + // removed the segments until the segment file is deleted + AggregateCompletionStage stage = CompletionStages.aggregateCompletionStage(); + // Then we need to delete the segment + for (int cacheSegment = 0; cacheSegment < addedCount; cacheSegment++) { + // We signal to complete the flowables, once complete the index is updated as we need to + // update the compactor + removedFlowables.get(cacheSegment).onComplete(); + Segment segment = removedSegments.get(cacheSegment); + stage.dependsOn( + segment.thenCompose(___ -> + // Now we free all entries in the index, this will include expired and removed entries + // Removed doesn't currently update free stats per ISPN-15246 - so we remove those as well + segment.root.publish((keyAndMetadataRecord, leafNode, fileProvider, timeService) -> { + compactor.free(leafNode.file, keyAndMetadataRecord.getHeader().totalLength()); + return null; + }).ignoreElements() + .toCompletionStage(null) + ) + .thenRun(segment::delete)); + } + stage.freeze() + .whenComplete((___, t) -> { + if (t != null) { + stageWhenComplete.completeExceptionally(t); + } else { + stageWhenComplete.complete(null); + } + }); + } catch (Throwable t) { + stageWhenComplete.completeExceptionally(t); + } + }); + } + static class Segment extends CompletableFuture implements Consumer, Action { final Index index; private final TemporaryTable temporaryTable; private final TreeMap> freeBlocks = new TreeMap<>(); private final ReadWriteLock rootLock = new ReentrantReadWriteLock(); - private final FileChannel indexFile; - private long indexFileSize; + private final int id; + private long indexFileSize = INDEX_FILE_HEADER_SIZE; private volatile IndexNode root; - private Segment(Index index, int id, TemporaryTable temporaryTable) throws IOException { + private Segment(Index index, int id, TemporaryTable temporaryTable) { this.index = index; this.temporaryTable = temporaryTable; - File indexFileFile = new File(index.indexDir.toFile(), "index." + id); - this.indexFile = new RandomAccessFile(indexFileFile, "rw").getChannel(); + this.id = id; // Just to init to empty root = IndexNode.emptyWithLeaves(this); } + public int getId() { + return id; + } + boolean load() throws IOException { int segmentMax = temporaryTable.getSegmentMax(); - indexFile.position(0); - ByteBuffer buffer = ByteBuffer.allocate(INDEX_FILE_HEADER_SIZE); - boolean loaded; - if (indexFile.size() >= INDEX_FILE_HEADER_SIZE && read(indexFile, buffer) - && buffer.getInt(0) == GRACEFULLY && buffer.getInt(4) == segmentMax) { - long rootOffset = buffer.getLong(8); - short rootOccupied = buffer.getShort(16); - long freeBlocksOffset = buffer.getLong(18); - root = new IndexNode(this, rootOffset, rootOccupied); - loadFreeBlocks(freeBlocksOffset); - indexFileSize = freeBlocksOffset; - loaded = true; - } else { - this.indexFile.truncate(0); - root = IndexNode.emptyWithLeaves(this); - loaded = false; - // reserve space for shutdown - indexFileSize = INDEX_FILE_HEADER_SIZE; + FileProvider.Handle handle = index.indexFileProvider.getFile(id); + try (handle) { + ByteBuffer buffer = ByteBuffer.allocate(INDEX_FILE_HEADER_SIZE); + boolean loaded; + if (handle.getFileSize() >= INDEX_FILE_HEADER_SIZE && read(handle, buffer, 0) + && buffer.getInt(0) == GRACEFULLY && buffer.getInt(4) == segmentMax) { + long rootOffset = buffer.getLong(8); + short rootOccupied = buffer.getShort(16); + long freeBlocksOffset = buffer.getLong(18); + root = new IndexNode(this, rootOffset, rootOccupied); + loadFreeBlocks(freeBlocksOffset); + indexFileSize = freeBlocksOffset; + loaded = true; + } else { + handle.truncate(0); + root = IndexNode.emptyWithLeaves(this); + loaded = false; + // reserve space for shutdown + indexFileSize = INDEX_FILE_HEADER_SIZE; + } + buffer.putInt(0, DIRTY); + buffer.position(0); + buffer.limit(4); + write(handle, buffer, 0); + + return loaded; } - buffer.putInt(0, DIRTY); - buffer.position(0); - buffer.limit(4); - indexFile.position(0); - write(indexFile, buffer); + } - return loaded; + void delete() { + // Empty segment is negative, so there is no file + if (id >= 0) { + log.tracef("Deleting file for index %s", id); + index.indexFileProvider.deleteFile(id); + } } void reset() throws IOException { - this.indexFile.truncate(0); - root = IndexNode.emptyWithLeaves(this); - // reserve space for shutdown - indexFileSize = INDEX_FILE_HEADER_SIZE; - ByteBuffer buffer = ByteBuffer.allocate(INDEX_FILE_HEADER_SIZE); - buffer.putInt(0, DIRTY); - buffer.position(0); - buffer.limit(4); - indexFile.position(0); + try (FileProvider.Handle handle = index.indexFileProvider.getFile(id)) { + handle.truncate(0); + root = IndexNode.emptyWithLeaves(this); + // reserve space for shutdown + indexFileSize = INDEX_FILE_HEADER_SIZE; + ByteBuffer buffer = ByteBuffer.allocate(INDEX_FILE_HEADER_SIZE); + buffer.putInt(0, DIRTY); + buffer.position(0); + buffer.limit(4); - write(indexFile, buffer); + write(handle, buffer, 0); + } } @Override @@ -507,7 +736,9 @@ public void accept(IndexRequest request) throws Throwable { switch (request.getType()) { case CLEAR: root = IndexNode.emptyWithLeaves(this); - indexFile.truncate(0); + try (FileProvider.Handle handle = index.indexFileProvider.getFile(id)) { + handle.truncate(0); + } indexFileSize = INDEX_FILE_HEADER_SIZE; freeBlocks.clear(); index.nonBlockingManager.complete(request, null); @@ -554,42 +785,44 @@ public void run() throws IOException { try { IndexSpace rootSpace = allocateIndexSpace(root.length()); root.store(rootSpace); - indexFile.position(indexFileSize); - ByteBuffer buffer = ByteBuffer.allocate(4); - buffer.putInt(0, freeBlocks.size()); - write(indexFile, buffer); - for (Map.Entry> entry : freeBlocks.entrySet()) { - List list = entry.getValue(); - int requiredSize = 8 + list.size() * 10; - buffer = buffer.capacity() < requiredSize ? ByteBuffer.allocate(requiredSize) : buffer; - buffer.position(0); - buffer.limit(requiredSize); - // TODO: change this to short - buffer.putInt(entry.getKey()); - buffer.putInt(list.size()); - for (IndexSpace space : list) { - buffer.putLong(space.offset); - buffer.putShort(space.length); + try (FileProvider.Handle handle = index.indexFileProvider.getFile(id)) { + ByteBuffer buffer = ByteBuffer.allocate(4); + buffer.putInt(0, freeBlocks.size()); + long offset = indexFileSize; + offset += write(handle, buffer, offset); + for (Map.Entry> entry : freeBlocks.entrySet()) { + List list = entry.getValue(); + int requiredSize = 8 + list.size() * 10; + buffer = buffer.capacity() < requiredSize ? ByteBuffer.allocate(requiredSize) : buffer; + buffer.position(0); + buffer.limit(requiredSize); + // TODO: change this to short + buffer.putInt(entry.getKey()); + buffer.putInt(list.size()); + for (IndexSpace space : list) { + buffer.putLong(space.offset); + buffer.putShort(space.length); + } + buffer.flip(); + offset += write(handle, buffer, offset); } + int headerWithoutMagic = INDEX_FILE_HEADER_SIZE - 4; + buffer = buffer.capacity() < headerWithoutMagic ? ByteBuffer.allocate(headerWithoutMagic) : buffer; + buffer.position(0); + // we need to set limit ahead, otherwise the putLong could throw IndexOutOfBoundsException + buffer.limit(headerWithoutMagic); + buffer.putInt(index.segments.length); + buffer.putLong(rootSpace.offset); + buffer.putShort(rootSpace.length); + buffer.putLong(indexFileSize); buffer.flip(); - write(indexFile, buffer); + write(handle, buffer, 4); + + buffer.position(0); + buffer.limit(4); + buffer.putInt(0, GRACEFULLY); + write(handle, buffer, 0); } - int headerWithoutMagic = INDEX_FILE_HEADER_SIZE - 8; - buffer = buffer.capacity() < headerWithoutMagic ? ByteBuffer.allocate(headerWithoutMagic) : buffer; - buffer.position(0); - // we need to set limit ahead, otherwise the putLong could throw IndexOutOfBoundsException - buffer.limit(headerWithoutMagic); - buffer.putLong(0, rootSpace.offset); - buffer.putShort(8, rootSpace.length); - buffer.putLong(10, indexFileSize); - indexFile.position(8); - write(indexFile, buffer); - buffer.position(0); - buffer.limit(8); - buffer.putInt(0, GRACEFULLY); - buffer.putInt(4, temporaryTable.getSegmentMax()); - indexFile.position(0); - write(indexFile, buffer); complete(null); } catch (Throwable t) { @@ -598,48 +831,62 @@ public void run() throws IOException { } private void loadFreeBlocks(long freeBlocksOffset) throws IOException { - indexFile.position(freeBlocksOffset); ByteBuffer buffer = ByteBuffer.allocate(8); buffer.limit(4); - if (!read(indexFile, buffer)) { - throw new IOException("Cannot read free blocks lists!"); - } - int numLists = buffer.getInt(0); - for (int i = 0; i < numLists; ++i) { - buffer.position(0); - buffer.limit(8); - if (!read(indexFile, buffer)) { + long offset = freeBlocksOffset; + try (FileProvider.Handle handle = index.indexFileProvider.getFile(id)) { + if (!read(handle, buffer, offset)) { throw new IOException("Cannot read free blocks lists!"); } - // TODO: change this to short - int blockLength = buffer.getInt(0); - assert blockLength <= Short.MAX_VALUE; - int listSize = buffer.getInt(4); - // Ignore any free block that had no entries as it adds time complexity to our lookup - if (listSize > 0) { - int requiredSize = 10 * listSize; - buffer = buffer.capacity() < requiredSize ? ByteBuffer.allocate(requiredSize) : buffer; + offset += 4; + int numLists = buffer.getInt(0); + for (int i = 0; i < numLists; ++i) { buffer.position(0); - buffer.limit(requiredSize); - if (!read(indexFile, buffer)) { + buffer.limit(8); + if (!read(handle, buffer, offset)) { throw new IOException("Cannot read free blocks lists!"); } - buffer.flip(); - ArrayList list = new ArrayList<>(listSize); - for (int j = 0; j < listSize; ++j) { - list.add(new IndexSpace(buffer.getLong(), buffer.getShort())); + offset += 8; + // TODO: change this to short + int blockLength = buffer.getInt(0); + assert blockLength <= Short.MAX_VALUE; + int listSize = buffer.getInt(4); + // Ignore any free block that had no entries as it adds time complexity to our lookup + if (listSize > 0) { + int requiredSize = 10 * listSize; + buffer = buffer.capacity() < requiredSize ? ByteBuffer.allocate(requiredSize) : buffer; + buffer.position(0); + buffer.limit(requiredSize); + if (!read(handle, buffer, offset)) { + throw new IOException("Cannot read free blocks lists!"); + } + offset += requiredSize; + buffer.flip(); + ArrayList list = new ArrayList<>(listSize); + for (int j = 0; j < listSize; ++j) { + list.add(new IndexSpace(buffer.getLong(), buffer.getShort())); + } + freeBlocks.put((short) blockLength, list); } - freeBlocks.put((short) blockLength, list); } } } - public FileChannel getIndexFile() { - return indexFile; + public FileProvider.Handle getIndexFile() throws IOException { + return index.indexFileProvider.getFile(id); + } + + public void forceIndexIfOpen(boolean metaData) throws IOException { + FileProvider.Handle handle = index.indexFileProvider.getFileIfOpen(id); + if (handle != null) { + try (handle) { + handle.force(metaData); + } + } } public FileProvider getFileProvider() { - return index.fileProvider; + return index.dataFileProvider; } public Compactor getCompactor() { @@ -701,8 +948,8 @@ void freeIndexSpace(long offset, short length) { freeBlocks.computeIfAbsent(length, k -> new ArrayList<>()).add(new IndexSpace(offset, length)); } else { indexFileSize -= length; - try { - indexFile.truncate(indexFileSize); + try (FileProvider.Handle handle = index.indexFileProvider.getFile(id)) { + handle.truncate(indexFileSize); } catch (IOException e) { log.cannotTruncateIndex(e); } @@ -754,7 +1001,27 @@ public String toString() { } Flowable publish(IntSet cacheSegments, boolean loadValues) { - return Flowable.fromArray(segments) - .concatMap(segment -> segment.root.publish(cacheSegments, loadValues)); + return Flowable.fromIterable(cacheSegments) + .concatMap(cacheSegment -> publish(cacheSegment, loadValues)); + } + + Flowable publish(int cacheSegment, boolean loadValues) { + var segment = segments[cacheSegment]; + if (segment.index.sizePerSegment.get(cacheSegment) == 0) { + return Flowable.empty(); + } + return segment.root.publish((keyAndMetadataRecord, leafNode, fileProvider, currentTime) -> { + long expiryTime = keyAndMetadataRecord.getHeader().expiryTime(); + // Ignore any key or value if it is expired or was removed + if (expiryTime > 0 && expiryTime < currentTime || keyAndMetadataRecord.getHeader().valueLength() <= 0) { + return null; + } + if (loadValues) { + log.tracef("Loading value record for leafNode: %s", leafNode); + + return leafNode.loadValue(keyAndMetadataRecord, fileProvider); + } + return keyAndMetadataRecord; + }); } } diff --git a/core/src/main/java/org/infinispan/persistence/sifs/IndexNode.java b/core/src/main/java/org/infinispan/persistence/sifs/IndexNode.java index 0f33ce98af3c..79d4b74d805a 100644 --- a/core/src/main/java/org/infinispan/persistence/sifs/IndexNode.java +++ b/core/src/main/java/org/infinispan/persistence/sifs/IndexNode.java @@ -3,7 +3,6 @@ import java.io.IOException; import java.lang.ref.SoftReference; import java.nio.ByteBuffer; -import java.nio.channels.FileChannel; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Arrays; @@ -14,13 +13,9 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; -import java.util.stream.Collectors; -import org.infinispan.commons.io.ByteBufferImpl; -import org.infinispan.commons.io.UnsignedNumeric; import org.infinispan.commons.time.TimeService; import org.infinispan.commons.util.ByRef; -import org.infinispan.commons.util.IntSet; import org.infinispan.commons.util.Util; import org.infinispan.reactive.FlowableCreate; import org.infinispan.util.logging.LogFactory; @@ -56,8 +51,8 @@ class IndexNode { // Prefix length (short) + keyNode length (short) + flag (byte) private static final int INNER_NODE_HEADER_SIZE = 5; private static final int INNER_NODE_REFERENCE_SIZE = 10; - // File size (int), Offset (int), numRecords (int), Cache Segment (int) - private static final int LEAF_NODE_REFERENCE_SIZE = 16; + // File size (int), Offset (int), numRecords (int) + private static final int LEAF_NODE_REFERENCE_SIZE = 12; public static final int RESERVED_SPACE = INNER_NODE_HEADER_SIZE + 2 * Math.max(INNER_NODE_REFERENCE_SIZE, LEAF_NODE_REFERENCE_SIZE); @@ -86,7 +81,10 @@ public enum RecordChange { this.offset = offset; this.occupiedSpace = occupiedSpace; - ByteBuffer buffer = loadBuffer(segment.getIndexFile(), offset, occupiedSpace); + ByteBuffer buffer; + try (FileProvider.Handle handle = segment.getIndexFile()) { + buffer = loadBuffer(handle, offset, occupiedSpace); + } prefix = new byte[buffer.getShort()]; buffer.get(prefix); @@ -106,7 +104,7 @@ public enum RecordChange { if ((flags & HAS_LEAVES) != 0) { leafNodes = new LeafNode[numKeyParts + 1]; for (int i = 0; i < numKeyParts + 1; ++i) { - leafNodes[i] = new LeafNode(buffer.getInt(), buffer.getInt(), buffer.getInt(), buffer.getInt()); + leafNodes[i] = new LeafNode(buffer.getInt(), buffer.getInt(), buffer.getInt()); } } else if ((flags & HAS_NODES) != 0) { innerNodes = new InnerNode[numKeyParts + 1]; @@ -122,14 +120,14 @@ public enum RecordChange { } } - private static ByteBuffer loadBuffer(FileChannel indexFile, long offset, int occupiedSpace) throws IOException { + private static ByteBuffer loadBuffer(FileProvider.Handle indexFile, long offset, int occupiedSpace) throws IOException { ByteBuffer buffer = ByteBuffer.allocate(occupiedSpace); int read = 0; do { int nowRead = indexFile.read(buffer, offset + read); if (nowRead < 0) { throw new IOException("Cannot read record [" + offset + ":" + occupiedSpace + "] (already read " - + read + "), file size is " + indexFile.size()); + + read + "), file size is " + indexFile.getFileSize()); } read += nowRead; } while (read < occupiedSpace); @@ -219,12 +217,13 @@ void store(Index.IndexSpace indexSpace) throws IOException { buffer.putInt(leafNode.file); buffer.putInt(leafNode.offset); buffer.putInt(leafNode.numRecords); - buffer.putInt(leafNode.cacheSegment); } } assert buffer.position() == buffer.limit() : "Buffer position: " + buffer.position() + " limit: " + buffer.limit(); buffer.flip(); - segment.getIndexFile().write(buffer, offset); + try (FileProvider.Handle handle = segment.getIndexFile()) { + handle.write(buffer, offset); + } if (log.isTraceEnabled()) { log.tracef("Persisted %08x (length %d, %d %s) to %d:%d", System.identityHashCode(this), length(), @@ -251,21 +250,21 @@ private static boolean entryKeyEqualsBuffer(EntryRecord headerAndKey, org.infini public enum ReadOperation { GET_RECORD { @Override - protected EntryRecord apply(LeafNode leafNode, org.infinispan.commons.io.ByteBuffer key, FileProvider fileProvider, TimeService timeService) throws IOException, IndexNodeOutdatedException { + protected EntryRecord apply(LeafNode leafNode, byte[] key, FileProvider fileProvider, TimeService timeService) throws IOException, IndexNodeOutdatedException { return leafNode.loadRecord(fileProvider, key, timeService); } }, GET_EXPIRED_RECORD { @Override - protected EntryRecord apply(LeafNode leafNode, org.infinispan.commons.io.ByteBuffer key, FileProvider fileProvider, TimeService timeService) throws IOException, IndexNodeOutdatedException { + protected EntryRecord apply(LeafNode leafNode, byte[] key, FileProvider fileProvider, TimeService timeService) throws IOException, IndexNodeOutdatedException { return leafNode.loadRecord(fileProvider, key, null); } }, GET_POSITION { @Override - protected EntryPosition apply(LeafNode leafNode, org.infinispan.commons.io.ByteBuffer key, FileProvider fileProvider, TimeService timeService) throws IOException, IndexNodeOutdatedException { + protected EntryPosition apply(LeafNode leafNode, byte[] key, FileProvider fileProvider, TimeService timeService) throws IOException, IndexNodeOutdatedException { EntryRecord hak = leafNode.loadHeaderAndKey(fileProvider); - if (entryKeyEqualsBuffer(hak, key)) { + if (Arrays.equals(hak.getKey(), key)) { if (hak.getHeader().expiryTime() > 0 && hak.getHeader().expiryTime() <= timeService.wallClockTime()) { if (log.isTraceEnabled()) { log.tracef("Found node on %d:%d but it is expired", leafNode.file, leafNode.offset); @@ -283,9 +282,9 @@ protected EntryPosition apply(LeafNode leafNode, org.infinispan.commons.io.ByteB }, GET_INFO { @Override - protected EntryInfo apply(LeafNode leafNode, org.infinispan.commons.io.ByteBuffer key, FileProvider fileProvider, TimeService timeService) throws IOException, IndexNodeOutdatedException { + protected EntryInfo apply(LeafNode leafNode, byte[] key, FileProvider fileProvider, TimeService timeService) throws IOException, IndexNodeOutdatedException { EntryRecord hak = leafNode.loadHeaderAndKey(fileProvider); - if (entryKeyEqualsBuffer(hak, key)) { + if (Arrays.equals(hak.getKey(), key)) { log.tracef("Found matching leafNode %s", leafNode); return leafNode; } else { @@ -297,7 +296,7 @@ protected EntryInfo apply(LeafNode leafNode, org.infinispan.commons.io.ByteBuffe } }; - protected abstract T apply(LeafNode leafNode, org.infinispan.commons.io.ByteBuffer key, FileProvider fileProvider, TimeService timeService) throws IOException, IndexNodeOutdatedException; + protected abstract T apply(LeafNode leafNode, byte[] key, FileProvider fileProvider, TimeService timeService) throws IOException, IndexNodeOutdatedException; } public static T applyOnLeaf(Index.Segment segment, int cacheSegment, byte[] indexKey, Lock rootLock, ReadOperation operation) throws IOException { @@ -325,9 +324,8 @@ public static T applyOnLeaf(Index.Segment segment, int cacheSegment, byte[] return null; } int insertionPoint = node.getInsertionPoint(indexKey); - int cacheSegmentBytesSize = UnsignedNumeric.sizeUnsignedInt(cacheSegment); - return operation.apply(node.leafNodes[insertionPoint], ByteBufferImpl.create(indexKey, cacheSegmentBytesSize, indexKey.length - cacheSegmentBytesSize), - segment.getFileProvider(), segment.getTimeService()); + return operation.apply(node.leafNodes[insertionPoint], indexKey, segment.getFileProvider(), + segment.getTimeService()); } catch (IndexNodeOutdatedException e) { try { if (attempts > 10) { @@ -392,7 +390,9 @@ private void updateFileOffsetInFile(int leafOffset, int newFile, int newOffset, buffer.flip(); - this.segment.getIndexFile().write(buffer, offset); + try (FileProvider.Handle handle = this.segment.getIndexFile()) { + handle.write(buffer, offset); + } } private static IndexNode findParentNode(IndexNode root, byte[] indexKey, Deque stack) throws IOException { @@ -411,8 +411,8 @@ private static IndexNode findParentNode(IndexNode root, byte[] indexKey, Deque

stack = new ArrayDeque<>(); IndexNode node = findParentNode(root, indexKey, stack); @@ -446,7 +446,7 @@ public static void setPosition(IndexNode root, IndexRequest request, OverwriteHo } } else { newRoot = IndexNode.emptyWithInnerNodes(root.segment).copyWith(0, 0, result.newNodes); - root.segment.getIndexFile().force(false); + root.segment.forceIndexIfOpen(false); if (log.isTraceEnabled()) { log.tracef("Setting new root %08x (index has grown)", System.identityHashCode(newRoot)); } @@ -643,7 +643,7 @@ private byte[] leftmostKey() throws IOException, IndexNodeOutdatedException { } else { for (LeafNode leafNode : leafNodes) { EntryRecord hak = leafNode.loadHeaderAndKey(segment.getFileProvider()); - if (hak.getKey() != null) return Index.toIndexKey(leafNode.cacheSegment, hak.getKey()); + if (hak.getKey() != null) return hak.getKey(); } } return null; @@ -658,7 +658,7 @@ private byte[] rightmostKey() throws IOException, IndexNodeOutdatedException { } else { for (int i = leafNodes.length - 1; i >= 0; --i) { EntryRecord hak = leafNodes[i].loadHeaderAndKey(segment.getFileProvider()); - if (hak.getKey() != null) return Index.toIndexKey(leafNodes[i].cacheSegment, hak.getKey()); + if (hak.getKey() != null) return hak.getKey(); } } return null; @@ -676,7 +676,7 @@ private IndexNode copyWith(IndexRequest request, int cacheSegment, byte[] indexK if (leafNodes.length == 0) { overwriteHook.setOverwritten(request, cacheSegment, false, -1, -1); if (overwriteHook.check(request, -1, -1)) { - return new IndexNode(segment, prefix, keyParts, new LeafNode[]{new LeafNode(file, offset, 1, cacheSegment)}); + return new IndexNode(segment, prefix, keyParts, new LeafNode[]{new LeafNode(file, offset, 1)}); } else { segment.getCompactor().free(file, size); return this; @@ -710,7 +710,7 @@ private IndexNode copyWith(IndexRequest request, int cacheSegment, byte[] indexK } catch (IndexNodeOutdatedException e) { throw new IllegalStateException("Index cannot be outdated for segment updater thread", e); } - byte[] oldIndexKey = Index.toIndexKey(oldLeafNode.cacheSegment, hak.getKey()); + byte[] oldIndexKey = hak.getKey(); int keyComp = compare(oldIndexKey, indexKey); Object objectKey = request.getKey(); if (keyComp == 0) { @@ -740,7 +740,7 @@ private IndexNode copyWith(IndexRequest request, int cacheSegment, byte[] indexK } lock.writeLock().lock(); try { - leafNodes[insertPart] = new LeafNode(file, offset, numRecords, cacheSegment); + leafNodes[insertPart] = new LeafNode(file, offset, numRecords); } finally { lock.writeLock().unlock(); } @@ -799,55 +799,36 @@ private IndexNode copyWith(IndexRequest request, int cacheSegment, byte[] indexK System.arraycopy(leafNodes, 0, newLeafNodes, 0, insertPart + 1); System.arraycopy(leafNodes, insertPart + 1, newLeafNodes, insertPart + 2, leafNodes.length - insertPart - 1); log.tracef("Creating new leafNode for %s at %d:%d", objectKey, file, offset); - newLeafNodes[insertPart + 1] = new LeafNode(file, offset, 1, cacheSegment); + newLeafNodes[insertPart + 1] = new LeafNode(file, offset, 1); } else { newKeyParts[insertPart] = substring(oldIndexKey, newPrefix.length, -keyComp); System.arraycopy(leafNodes, 0, newLeafNodes, 0, insertPart); System.arraycopy(leafNodes, insertPart, newLeafNodes, insertPart + 1, leafNodes.length - insertPart); log.tracef("Creating new leafNode for %s at %d:%d", objectKey, file, offset); - newLeafNodes[insertPart] = new LeafNode(file, offset, 1, cacheSegment); + newLeafNodes[insertPart] = new LeafNode(file, offset, 1); } } return new IndexNode(segment, newPrefix, newKeyParts, newLeafNodes); } - private int getIterationPoint(byte[] key, int cacheSegment) { - int comp = compare(key, prefix, prefix.length); - int insertionPoint; - if (comp > 0) { - insertionPoint = 0; - } else if (comp < 0) { - insertionPoint = keyParts.length; - } else { - byte[] keyPostfix = substring(key, prefix.length, key.length); - insertionPoint = Arrays.binarySearch(keyParts, keyPostfix, REVERSED_COMPARE_TO); - if (insertionPoint < 0) { - insertionPoint = -insertionPoint - 1; - } else { - int cacheSegmentToUse = cacheSegment < 0 ? UnsignedNumeric.readUnsignedInt(key, 0) : cacheSegment; - if (UnsignedNumeric.sizeUnsignedInt(cacheSegmentToUse) < key.length) { - // When the length is bigger than a cache segment, that means the index prefix is a specific key and if it - // is equal we have to skip two spaces - // Example: - // KeyParts - // 84 = {byte[12]@9221} [-100, 1, -104, 1, 2, -118, 1, 5, 10, 3, 40, -71] - // 85 = {byte[12]@9222} [-100, 1, -104, 1, 2, -118, 1, 5, 10, 3, 40, -60] - // 86 = {byte[13]@9223} [-100, 1, -104, 1, 2, -118, 1, 5, 10, 3, 40, -60, 14] - // 87 = {byte[12]@9224} [-100, 1, -104, 1, 2, -118, 1, 5, 10, 3, 40, -54] - // 88 = {byte[12]@9225} [-100, 1, -104, 1, 2, -118, 1, 5, 10, 3, 40, -48] - // Segment Prefix - // {byte[13] [-100, 1, -104, 1, 2, -118, 1, 5, 10, 3, 40, -60, 14] - // The actual value is stored at 87 in this case per `getInsertionPoint` so we need to skip to 88 - // CacheSegment is -1 for an innerNode because we have to find where in the leaf node the value is - // CacheSegment is > 0 for a leafNode - insertionPoint += cacheSegment < 0 ? 1 : 2; - } - } - } - return insertionPoint; + private int getInsertionPoint(byte[] key) { + return getInsertionPoint(key, false); } - private int getInsertionPoint(byte[] key) { + /** + * Finds where an entry would be inserted into the node. It will be 0th position if this key is before the prefix and + * the length of the array if it does not start with the given prefix. + *

+ * When the prefix matches the first bytes of the key the point returned will be where either the key is or the key + * before where it would belong if the key is not present. + *

+ * It is possible to instead always return the next position that is greater than the key by passing true for + * addExtraOnPrefixMatch. This can be useful to find the next key after a given one for doing iteration. + * @param key the key to find its position in the nodes + * @param addExtraOnPrefixMatch whether the returned position is always greater than the provided key + * @return the position in the nodes + */ + private int getInsertionPoint(byte[] key, boolean addExtraOnPrefixMatch) { int comp = compare(key, prefix, prefix.length); int insertionPoint; if (comp > 0) { @@ -858,9 +839,9 @@ private int getInsertionPoint(byte[] key) { byte[] keyPostfix = substring(key, prefix.length, key.length); insertionPoint = Arrays.binarySearch(keyParts, keyPostfix, REVERSED_COMPARE_TO); if (insertionPoint < 0) { - insertionPoint = -insertionPoint - 1; + insertionPoint = -insertionPoint - 1 + (addExtraOnPrefixMatch ? 1 : 0); } else { - insertionPoint++; // identical elements must go to the right + insertionPoint += addExtraOnPrefixMatch ? 2 : 1; // identical elements must go to the right } } @@ -1113,12 +1094,12 @@ IndexNode getIndexNode(Index.Segment segment) throws IOException { } } - private static class LeafNode extends EntryInfo { + static class LeafNode extends EntryInfo { private static final LeafNode[] EMPTY_ARRAY = new LeafNode[0]; private volatile SoftReference keyReference; - LeafNode(int file, int offset, int numRecords, int cacheSegment) { - super(file, offset, numRecords, cacheSegment); + LeafNode(int file, int offset, int numRecords) { + super(file, offset, numRecords); } public EntryRecord loadHeaderAndKey(FileProvider fileProvider) throws IOException, IndexNodeOutdatedException { @@ -1162,7 +1143,7 @@ private EntryRecord getHeaderAndKey(FileProvider fileProvider, FileProvider.Hand return headerAndKey; } - public EntryRecord loadRecord(FileProvider fileProvider, org.infinispan.commons.io.ByteBuffer key, TimeService timeService) throws IOException, IndexNodeOutdatedException { + public EntryRecord loadRecord(FileProvider fileProvider, byte[] key, TimeService timeService) throws IOException, IndexNodeOutdatedException { FileProvider.Handle handle = fileProvider.getFile(file); int readOffset = offset < 0 ? ~offset : offset; if (handle == null) { @@ -1171,7 +1152,7 @@ public EntryRecord loadRecord(FileProvider fileProvider, org.infinispan.commons. try { boolean trace = log.isTraceEnabled(); EntryRecord headerAndKey = getHeaderAndKey(fileProvider, handle); - if (key != null && !entryKeyEqualsBuffer(headerAndKey, key)) { + if (key != null && !Arrays.equals(headerAndKey.getKey(), key)) { if (trace) { log.trace("Key on " + file + ":" + readOffset + " not matched."); } @@ -1197,11 +1178,26 @@ public EntryRecord loadRecord(FileProvider fileProvider, org.infinispan.commons. handle.close(); } } - } - private static class IndexNodeOutdatedException extends Exception { - IndexNodeOutdatedException(String message) { - super(message); + /** + * Loads the value and internal metadata for a given header an existing key entry record. Note this method will + * not cache the value in this EntryRecord if it isn't already present. + * This method will not check expiration or if the value was removed as it is the callers duty + * @param headerAndKey existing EntryRecord that contains at least the key and header + * @param fileProvider file provider to use to read the value as needed + * @return an EntryRecord that contains the key, metadata and value. + * @throws IOException if there was an issue reading the entry + * @throws IndexNodeOutdatedException if the file has changed since reading the header and key + */ + public EntryRecord loadValue(EntryRecord headerAndKey, FileProvider fileProvider) throws IOException, IndexNodeOutdatedException { + FileProvider.Handle handle = fileProvider.getFile(file); + int readOffset = offset < 0 ? ~offset : offset; + if (handle == null) { + throw new IndexNodeOutdatedException(file + ":" + readOffset); + } + try (handle) { + return headerAndKey.loadMetadataAndValue(handle, readOffset, false); + } } } @@ -1211,7 +1207,7 @@ public String toString() { for (int i = 0; i <= keyParts.length; ++i) { sb.append('\n'); if (leafNodes != null && i < leafNodes.length) { - sb.append(" [").append(leafNodes[i].file).append(':').append(leafNodes[i].offset).append(':').append(leafNodes[i].cacheSegment).append("] "); + sb.append(" [").append(leafNodes[i].file).append(':').append(leafNodes[i].offset).append("] "); } else { sb.append(" [").append(innerNodes[i].offset).append(':').append(innerNodes[i].length).append("] "); } @@ -1223,69 +1219,66 @@ public String toString() { return sb.toString(); } - Flowable publish(IntSet cacheSegments, boolean loadValues) { - long currentTime = segment.getTimeService().wallClockTime(); + interface PublishFunction { + /** + * Allows for a entry in the index to be converted to a desired value. This method is invoked for every + * entry in the index, which will include removed and expired values. This will only be invoked once per unique + * key in this index. + * @param keyAndMetadataRecord the record that is guaranteed to contain the key and index metadata + * @param leafNode the leafNode that points to the data file + * @param fileProvider the data fileProvider which can be used to read more information + * @param currentTime the current time to use for expiration + * @return converted value to return via the Publisher + */ + R apply(EntryRecord keyAndMetadataRecord, LeafNode leafNode, FileProvider fileProvider, long currentTime) + throws IOException, IndexNodeOutdatedException; + } - int cacheSegmentSize = cacheSegments.size(); - if (cacheSegmentSize == 0) { - return Flowable.empty(); - } + Flowable publish(PublishFunction publishFunction) { + long currentTime = segment.getTimeService().wallClockTime(); - // Needs defer as we mutate the deque so publisher can be subscribed to multiple times + // Needs defer as we mutate the lastRetrievedKey so inner FlowableCreate can be subscribed to multiple times return Flowable.defer(() -> { - // First sort all the cacheSegments by their unsigned numeric byte[] values. - // This allows us to start at the left most node, and we can iterate within the nodes if the cacheSegments are - // contiguous in the data - Deque sortedSegmentPrefixes = cacheSegments.intStream() - .filter(cacheSegment -> segment.index.sizePerSegment.get(cacheSegment) != 0) - .mapToObj(cacheSegment -> { - byte[] segmentPrefix = new byte[UnsignedNumeric.sizeUnsignedInt(cacheSegment)]; - UnsignedNumeric.writeUnsignedInt(segmentPrefix, 0, cacheSegment); - return segmentPrefix; - }).sorted(REVERSED_COMPARE_TO) - .collect(Collectors.toCollection(ArrayDeque::new)); - if (sortedSegmentPrefixes.isEmpty()) { - return Flowable.empty(); - } + ByRef lastRetrievedKey = new ByRef<>(Util.EMPTY_BYTE_ARRAY); return new FlowableCreate<>(emitter -> { - // Set to true in 3 different cases: cacheSegment didn't map to next entry, emitter has no more requests or cancelled + // Set to true in 2 different cases: emitter has no more requests or cancelled ByRef.Boolean done = new ByRef.Boolean(false); do { // Reset so we can loop done.set(false); - recursiveNode(this, segment, sortedSegmentPrefixes, emitter, loadValues, currentTime, new ByRef.Boolean(false), done, false); + recursiveNode(publishFunction, this, segment, lastRetrievedKey, emitter, currentTime, + new ByRef.Boolean(false), done); // This handles two of the done cases - in which case we can't continue if (emitter.requested() == 0 || emitter.isCancelled()) { return; } - } while (done.get() && !sortedSegmentPrefixes.isEmpty()); + } while (done.get() && lastRetrievedKey.get() != null); emitter.onComplete(); }, BackpressureStrategy.ERROR); }); } - void recursiveNode(IndexNode node, Index.Segment segment, Deque segmentPrefixes, FlowableEmitter emitter, - boolean loadValues, long currentTime, ByRef.Boolean foundData, ByRef.Boolean done, boolean firstNodeAttempted) throws IOException { + void recursiveNode(PublishFunction publishFunction, IndexNode node, Index.Segment segment, ByRef lastRetrievedKey, FlowableEmitter emitter, + long currentTime, ByRef.Boolean foundData, ByRef.Boolean done) throws IOException { Lock readLock = node.lock.readLock(); readLock.lock(); try { byte[] previousKey = null; - int previousSegment = -1; if (node.innerNodes != null) { - final int point = foundData.get() ? 0 : node.getIterationPoint(segmentPrefixes.getFirst(), -1); + byte[] lastKey = lastRetrievedKey.get(); + final int point = foundData.get() ? 0 : node.getInsertionPoint(lastKey, false); // Need to search all inner nodes starting from that point until we hit the last entry for the segment - for (int i = point; !segmentPrefixes.isEmpty() && i < node.innerNodes.length && !done.get(); ++i) { - recursiveNode(node.innerNodes[i].getIndexNode(segment), segment, segmentPrefixes, emitter, loadValues, - currentTime, foundData, done, i == point); + for (int i = point; i < node.innerNodes.length && !done.get(); ++i) { + recursiveNode(publishFunction, node.innerNodes[i].getIndexNode(segment), segment, lastRetrievedKey, emitter, + currentTime, foundData, done); } } else if (node.leafNodes != null) { int suggestedIteration; - byte[] segmentPrefix = segmentPrefixes.getFirst(); - int cacheSegment = UnsignedNumeric.readUnsignedInt(segmentPrefix, 0); + byte[] lastKey = lastRetrievedKey.get(); boolean firstData = !foundData.get(); if (firstData) { - suggestedIteration = node.getIterationPoint(segmentPrefix, cacheSegment); + suggestedIteration = node.getInsertionPoint(lastKey, lastKey.length != 0); foundData.set(true); } else { suggestedIteration = 0; @@ -1293,84 +1286,35 @@ void recursiveNode(IndexNode node, Index.Segment segment, Deque segmentP for (int i = suggestedIteration; i < node.leafNodes.length; ++i) { LeafNode leafNode = node.leafNodes[i]; - if (leafNode.cacheSegment != cacheSegment) { - // The suggestion may be off by 1 if the page index prefix is longer than the segment but equal - if (i == suggestedIteration && firstData - && segmentPrefix.length == UnsignedNumeric.sizeUnsignedInt(cacheSegment)) { - - // No entry for the given segment, make sure to try next segment - if (i != node.leafNodes.length - 1 - && (i == node.keyParts.length || - (i < node.keyParts.length && - compare(node.keyParts[i], segmentPrefix, Math.min(segmentPrefix.length, node.keyParts[i].length)) == 0))) - continue; - - // The cache segment does not map to the current innerNode, we are at the end of the leafNodes, - // and this is the first innerNode attempted. We need to also check the first leaf of the next innerNode if present. - if (i == node.leafNodes.length - 1 && firstNodeAttempted) { - return; - } - } - segmentPrefixes.removeFirst(); - - // If the data maps to the next segment in our ordered queue, we can continue reading, - // otherwise we end and the retry will kick in - segmentPrefix = segmentPrefixes.peekFirst(); - if (segmentPrefix != null) { - cacheSegment = UnsignedNumeric.readUnsignedInt(segmentPrefix, 0); - } - // Next cacheSegment doesn't match either, thus we have to retry with the next prefix - // Note that if segmentPrefix is null above, this will always be true - if (leafNode.cacheSegment != cacheSegment) { - done.set(true); - return; - } - } EntryRecord record; try { - if (loadValues) { - log.tracef("Loading record for leafNode: %s", leafNode); - record = leafNode.loadRecord(segment.getFileProvider(), null, segment.getTimeService()); - } else { - log.tracef("Loading header and key for leafNode: %s", leafNode); - record = leafNode.getHeaderAndKey(segment.getFileProvider(), null); - } - } catch (IndexNodeOutdatedException e) { - // Current key was outdated, we have to try from the previous entry we saw (note it is skipped) - if (previousKey != null) { - byte[] currentIndexKey = Index.toIndexKey(previousSegment, previousKey); - segmentPrefixes.removeFirst(); - segmentPrefixes.addFirst(currentIndexKey); - } - done.set(true); - return; - } + // We load the header and key first only, just in case it is a duplicate + log.tracef("Loading header and key for leafNode: %s", leafNode); + record = leafNode.getHeaderAndKey(segment.getFileProvider(), null); - if (record != null && record.getHeader().valueLength() > 0) { - // It is possible that the very first looked up entry was a previously seen value and if so + // It is possible that the very first looked up entry was a previously seen key and if so // we must skip it if it is equal to not return it twice. - // The current segmentPrefix will match the element's key bytes excluding the segment bytes if (firstData && i == suggestedIteration) { int keyLength = record.getHeader().keyLength(); - int lengthDiff = segmentPrefix.length - keyLength; - if (lengthDiff > 0) { + int lengthDiff = lastKey.length - keyLength; + if (lengthDiff == 0) { byte[] keyArray = record.getKey(); - if (Util.arraysEqual(keyArray, 0, keyArray.length, segmentPrefix, lengthDiff, segmentPrefix.length)) { + if (Arrays.equals(keyArray, lastKey)) { continue; } } } - long expiryTime = record.getHeader().expiryTime(); - if (expiryTime < 0 || expiryTime > currentTime) { - emitter.onNext(record); + + R value = publishFunction.apply(record, leafNode, segment.getFileProvider(), currentTime); + + if (value != null) { + emitter.onNext(value); if (emitter.requested() == 0) { // Store the current key as the next prefix when we can't retrieve more values, so // the next request will get the next value after this one - byte[] currentIndexKey = Index.toIndexKey(cacheSegment, record.getKey()); - segmentPrefixes.removeFirst(); - segmentPrefixes.addFirst(currentIndexKey); + lastRetrievedKey.set(record.getKey()); done.set(true); return; } else if (emitter.isCancelled()) { @@ -1378,18 +1322,21 @@ record = leafNode.getHeaderAndKey(segment.getFileProvider(), null); return; } } - previousKey = record.getKey(); - previousSegment = cacheSegment; + } catch (IndexNodeOutdatedException e) { + // Current key was outdated, we have to try from the previous entry we saw (note it is skipped) + if (previousKey != null) { + lastRetrievedKey.set(previousKey); + } + done.set(true); + return; } } // We are continuing with the next innerNode, save the previous key, just in case we get an outdated // exception on the first entry if (previousKey != null) { - byte[] currentIndexKey = Index.toIndexKey(previousSegment, previousKey); - segmentPrefixes.removeFirst(); - segmentPrefixes.addFirst(currentIndexKey); + lastRetrievedKey.set(previousKey); } } } finally { diff --git a/core/src/main/java/org/infinispan/persistence/sifs/IndexNodeOutdatedException.java b/core/src/main/java/org/infinispan/persistence/sifs/IndexNodeOutdatedException.java new file mode 100644 index 000000000000..70246c7e9ba6 --- /dev/null +++ b/core/src/main/java/org/infinispan/persistence/sifs/IndexNodeOutdatedException.java @@ -0,0 +1,7 @@ +package org.infinispan.persistence.sifs; + +class IndexNodeOutdatedException extends Exception { + IndexNodeOutdatedException(String message) { + super(message); + } +} diff --git a/core/src/main/java/org/infinispan/persistence/sifs/LogAppender.java b/core/src/main/java/org/infinispan/persistence/sifs/LogAppender.java index a2aee10ce572..8b42c62391d7 100644 --- a/core/src/main/java/org/infinispan/persistence/sifs/LogAppender.java +++ b/core/src/main/java/org/infinispan/persistence/sifs/LogAppender.java @@ -95,7 +95,7 @@ public synchronized void start(Executor executor) { Util.close(logFile); // add the current appended file - note this method will fail if it is already present, which will // happen if there are some free entries - compactor.addFreeFile(logFile.fileId, (int) fileProvider.getFileSize(logFile.fileId), 0, nextExpirationTime); + compactor.addLogFileOnShutdown(logFile.fileId, nextExpirationTime); logFile = null; } }); diff --git a/core/src/main/java/org/infinispan/persistence/sifs/NonBlockingSoftIndexFileStore.java b/core/src/main/java/org/infinispan/persistence/sifs/NonBlockingSoftIndexFileStore.java index b1c75273edf7..df14205c282b 100644 --- a/core/src/main/java/org/infinispan/persistence/sifs/NonBlockingSoftIndexFileStore.java +++ b/core/src/main/java/org/infinispan/persistence/sifs/NonBlockingSoftIndexFileStore.java @@ -163,13 +163,13 @@ public Set characteristics() { @Override public CompletionStage addSegments(IntSet segments) { temporaryTable.addSegments(segments); - return CompletableFutures.completedNull(); + return index.addSegments(segments); } @Override public CompletionStage removeSegments(IntSet segments) { temporaryTable.removeSegments(segments); - return CompletableFutures.completedNull(); + return index.removeSegments(segments); } @Override @@ -193,15 +193,20 @@ public CompletionStage start(InitializationContext ctx) { temporaryTable = new TemporaryTable(cacheSegments); temporaryTable.addSegments(IntSets.immutableRangeSet(cacheConfig.clustering().hash().numSegments())); - fileProvider = new FileProvider(getDataLocation(), configuration.openFilesLimit(), PREFIX_LATEST, + int maxOpenFiles = configuration.openFilesLimit(); + // Use index files between 1 and cacheSegments + int maxOpenIndexFiles = Math.min(Math.max(maxOpenFiles / 10, 1), cacheSegments); + int maxOpenDataFiles = maxOpenFiles - maxOpenIndexFiles; + + fileProvider = new FileProvider(getDataLocation(), maxOpenDataFiles, PREFIX_LATEST, configuration.maxFileSize()); compactor = new Compactor(ctx.getNonBlockingManager(), fileProvider, temporaryTable, marshaller, timeService, keyPartitioner, configuration.maxFileSize(), configuration.compactionThreshold(), blockingManager.asExecutor("sifs-compactor")); try { - index = new Index(ctx.getNonBlockingManager(), fileProvider, getIndexLocation(), configuration.indexSegments(), - cacheSegments, configuration.minNodeSize(), configuration.maxNodeSize(), temporaryTable, compactor, - timeService); + index = new Index(ctx.getNonBlockingManager(), fileProvider, getIndexLocation(), cacheSegments, + configuration.minNodeSize(), configuration.maxNodeSize(), temporaryTable, compactor, + timeService, blockingManager.asExecutor("sifs-index"), maxOpenIndexFiles); } catch (IOException e) { throw log.cannotOpenIndex(configuration.indexLocation(), e); } @@ -446,7 +451,7 @@ protected boolean isSeqIdOld(long seqId, int segment, Object key, byte[] seriali protected void startIndex() { // this call is extracted for better testability - index.start(blockingManager.asExecutor("sifs-index")); + index.start(); } @Override diff --git a/core/src/main/java/org/infinispan/persistence/sifs/configuration/IndexConfiguration.java b/core/src/main/java/org/infinispan/persistence/sifs/configuration/IndexConfiguration.java index 44ad08775598..03b2b366ce75 100644 --- a/core/src/main/java/org/infinispan/persistence/sifs/configuration/IndexConfiguration.java +++ b/core/src/main/java/org/infinispan/persistence/sifs/configuration/IndexConfiguration.java @@ -34,6 +34,10 @@ public void setLocation(String location) { attributes.attribute(INDEX_LOCATION).set(location); } + /** + * This is no longer used as we create an index file per cache segment instead + */ + @Deprecated(since = "15.0", forRemoval = true) public int indexSegments() { return attributes.attribute(INDEX_SEGMENTS).get(); } diff --git a/core/src/main/java/org/infinispan/persistence/sifs/configuration/IndexConfigurationBuilder.java b/core/src/main/java/org/infinispan/persistence/sifs/configuration/IndexConfigurationBuilder.java index 6dbc986e222e..c16f83500613 100644 --- a/core/src/main/java/org/infinispan/persistence/sifs/configuration/IndexConfigurationBuilder.java +++ b/core/src/main/java/org/infinispan/persistence/sifs/configuration/IndexConfigurationBuilder.java @@ -35,6 +35,7 @@ public IndexConfigurationBuilder indexLocation(String indexLocation) { return this; } + @Deprecated(since = "15.0") public IndexConfigurationBuilder indexSegments(int indexSegments) { attributes.attribute(INDEX_SEGMENTS).set(indexSegments); return this; diff --git a/core/src/main/java/org/infinispan/persistence/sifs/configuration/SoftIndexFileStoreConfiguration.java b/core/src/main/java/org/infinispan/persistence/sifs/configuration/SoftIndexFileStoreConfiguration.java index bedd9fa42e52..9de324d8f351 100644 --- a/core/src/main/java/org/infinispan/persistence/sifs/configuration/SoftIndexFileStoreConfiguration.java +++ b/core/src/main/java/org/infinispan/persistence/sifs/configuration/SoftIndexFileStoreConfiguration.java @@ -42,6 +42,10 @@ public String indexLocation() { return index.indexLocation(); } + /** + * This is no longer used as we create an index file per cache segment instead + */ + @Deprecated(since = "15.0", forRemoval = true) public int indexSegments() { return index.indexSegments(); } @@ -66,6 +70,15 @@ public boolean syncWrites() { return data.syncWrites(); } + /** + * The maximum number of files that will be open at a given time for all the data and index files, which does + * not include compactor and current log file (which will always be 2). + * Note that the number of data files is effectively unlimited, where as we have an index file per segment. + *

+ * Index files will reserve 1/10th of the open files, with a minimum value of 1 and a maximum equal to the + * number of cache segments. + * @return How many open files SIFS will utilize + */ public int openFilesLimit() { return attributes.attribute(OPEN_FILES_LIMIT).get(); } diff --git a/core/src/main/java/org/infinispan/persistence/sifs/configuration/SoftIndexFileStoreConfigurationBuilder.java b/core/src/main/java/org/infinispan/persistence/sifs/configuration/SoftIndexFileStoreConfigurationBuilder.java index 89cbdf12e880..edb98d8093fd 100644 --- a/core/src/main/java/org/infinispan/persistence/sifs/configuration/SoftIndexFileStoreConfigurationBuilder.java +++ b/core/src/main/java/org/infinispan/persistence/sifs/configuration/SoftIndexFileStoreConfigurationBuilder.java @@ -61,10 +61,9 @@ public SoftIndexFileStoreConfigurationBuilder indexLocation (String indexLocatio } /** - * Number of index segment files. Increasing this value improves throughput but requires more threads to be spawned. - *

- * Defaults to 16. + * No longer used, the number of index segments is determined solely by the number of cache segments. */ + @Deprecated(since = "15.0") public SoftIndexFileStoreConfigurationBuilder indexSegments ( int indexSegments){ index.indexSegments(indexSegments); return this; diff --git a/core/src/main/java/org/infinispan/persistence/support/DelegatingPersistenceManager.java b/core/src/main/java/org/infinispan/persistence/support/DelegatingPersistenceManager.java index 578edaf7bc04..3482592ce4b7 100644 --- a/core/src/main/java/org/infinispan/persistence/support/DelegatingPersistenceManager.java +++ b/core/src/main/java/org/infinispan/persistence/support/DelegatingPersistenceManager.java @@ -11,6 +11,7 @@ import org.infinispan.commons.api.Lifecycle; import org.infinispan.commons.util.IntSet; import org.infinispan.configuration.cache.StoreConfiguration; +import org.infinispan.container.entries.MVCCEntry; import org.infinispan.context.InvocationContext; import org.infinispan.context.impl.TxInvocationContext; import org.infinispan.factories.ComponentRegistry; @@ -23,6 +24,7 @@ import org.infinispan.persistence.spi.MarshallableEntry; import org.infinispan.persistence.spi.PersistenceException; import org.infinispan.transaction.impl.AbstractCacheTransaction; +import org.infinispan.util.function.TriPredicate; import org.reactivestreams.Publisher; import io.reactivex.rxjava3.core.Flowable; @@ -200,7 +202,7 @@ public CompletionStage writeEntries(Iterable performBatch(TxInvocationContext invocationContext, - BiPredicate commandKeyPredicate) { + TriPredicate> commandKeyPredicate) { return persistenceManager.performBatch(invocationContext, commandKeyPredicate); } diff --git a/core/src/main/java/org/infinispan/util/function/TriPredicate.java b/core/src/main/java/org/infinispan/util/function/TriPredicate.java new file mode 100644 index 000000000000..aaf736fbedf6 --- /dev/null +++ b/core/src/main/java/org/infinispan/util/function/TriPredicate.java @@ -0,0 +1,34 @@ +package org.infinispan.util.function; + +import java.util.function.Predicate; +import java.util.function.BiPredicate; + +/** + * Represents a predicate (boolean-valued function) of three arguments. This is + * the three-arity specialization of {@link Predicate}. + * + *

This is a functional interface + * whose functional method is {@link #test(Object, Object, Object)}. + * + * @param the type of the first argument to the predicate + * @param the type of the second argument the predicate + * @param the type of the third argument the predicate + * + * @see Predicate + * @see BiPredicate + * @since 15.0 + */ +@FunctionalInterface +public interface TriPredicate { + + /** + * Evaluates this predicate on the given arguments. + * + * @param t the first input argument + * @param u the second input argument + * @param v the third input argument + * @return {@code true} if the input arguments match the predicate, + * otherwise {@code false} + */ + boolean test(T t, U u, V v); +} diff --git a/core/src/main/java/org/infinispan/xsite/BackupSenderImpl.java b/core/src/main/java/org/infinispan/xsite/BackupSenderImpl.java index 991437df1287..02e5b9485971 100644 --- a/core/src/main/java/org/infinispan/xsite/BackupSenderImpl.java +++ b/core/src/main/java/org/infinispan/xsite/BackupSenderImpl.java @@ -250,7 +250,7 @@ private List filterModifications(List modifications, // in a replicable and another time in a non-replicable way for (ListIterator it = modifications.listIterator(modifications.size()); it.hasPrevious(); ) { WriteCommand writeCommand = it.previous(); - if (!writeCommand.isSuccessful() || writeCommand.hasAnyFlag(FlagBitSets.SKIP_XSITE_BACKUP)) { + if (!writeCommand.shouldReplicate(null, true) || writeCommand.hasAnyFlag(FlagBitSets.SKIP_XSITE_BACKUP)) { continue; } // Note: ClearCommand should be replicated out of transaction diff --git a/core/src/main/resources/schema/infinispan-config-14.0.xsd b/core/src/main/resources/schema/infinispan-config-14.0.xsd index 1cb0e787b0d7..24bcd49afce6 100644 --- a/core/src/main/resources/schema/infinispan-config-14.0.xsd +++ b/core/src/main/resources/schema/infinispan-config-14.0.xsd @@ -2052,7 +2052,8 @@ - Max number of data files opened for reading (current log file, compaction output and index segments are not included here). + Max number of data and index files opened for reading (current log file and compaction output are not included here - always uses one each). + Index files will use 1/10th of list limit with a minimum of 1 and a maximum equal to the number of cache segments. @@ -2112,7 +2113,8 @@ - Specifies the number of index segment files. + Deprecated since 15.0 which ignores the property. + This value is ignored as we create an index file per cache segment instead. diff --git a/core/src/test/java/org/infinispan/api/mvcc/PutForExternalReadTest.java b/core/src/test/java/org/infinispan/api/mvcc/PutForExternalReadTest.java index 4b6eaa99abbb..7332a8e6c5b0 100644 --- a/core/src/test/java/org/infinispan/api/mvcc/PutForExternalReadTest.java +++ b/core/src/test/java/org/infinispan/api/mvcc/PutForExternalReadTest.java @@ -167,8 +167,11 @@ public void testTxSuspension() throws Exception { } public void testExceptionSuppression() throws Exception { - Cache cache1 = cache(0, CACHE_NAME); - Cache cache2 = cache(1, CACHE_NAME); + Cache cache1 = cache(0, CACHE_NAME); + Cache cache2 = cache(1, CACHE_NAME); + + // Need to use a MagicKey so the interceptor is always installed on the primary + MagicKey key = new MagicKey(cache1); assertTrue(cache1.getAdvancedCache().getAsyncInterceptorChain().addInterceptorBefore(new BaseAsyncInterceptor() { @Override @@ -199,7 +202,7 @@ public Object visitCommand(InvocationContext ctx, VisitableCommand command) assertNull("Should have cleaned up", cache1.getAdvancedCache().getDataContainer().get(key)); assertNull("Should have cleaned up", cache2.get(key)); // scattered cache leaves tombstone - InternalCacheEntry cache2Entry = cache2.getAdvancedCache().getDataContainer().get(key); + InternalCacheEntry cache2Entry = cache2.getAdvancedCache().getDataContainer().get(key); assertTrue("Should have cleaned up", cache2Entry == null || cache2Entry.getValue() == null); // should not barf diff --git a/core/src/test/java/org/infinispan/eviction/impl/EvictionWithConcurrentOperationsTest.java b/core/src/test/java/org/infinispan/eviction/impl/EvictionWithConcurrentOperationsTest.java index 259106a9804e..75b1ecfd3d98 100644 --- a/core/src/test/java/org/infinispan/eviction/impl/EvictionWithConcurrentOperationsTest.java +++ b/core/src/test/java/org/infinispan/eviction/impl/EvictionWithConcurrentOperationsTest.java @@ -496,13 +496,8 @@ protected void assertInMemory(Object key, Object value) { assertEquals("Wrong value for key " + key + " in data container", value, entry.getValue()); WaitNonBlockingStore loader = TestingUtil.getFirstStoreWait(cache); - if (passivation) { - // With passivation the entry must not exist in the store - // but the removal is sometimes delayed - PersistenceManager pm = TestingUtil.extractComponent(cache, PersistenceManager.class); - assertNull(join(pm.loadFromAllStores(key, true, true))); - eventuallyEquals(null, () -> loader.loadEntry(key)); - } else { + // The entry may or may not be on disk with passivation + if (!passivation) { MarshallableEntry entryLoaded = loader.loadEntry(key); assertEquals("Wrong value for key " + key + " in cache loader", value, extractValue(entryLoaded)); } diff --git a/core/src/test/java/org/infinispan/eviction/impl/EvictionWithPassivationAndConcurrentOperationsTest.java b/core/src/test/java/org/infinispan/eviction/impl/EvictionWithPassivationAndConcurrentOperationsTest.java index 1c9edbea7f74..f2534583fc34 100644 --- a/core/src/test/java/org/infinispan/eviction/impl/EvictionWithPassivationAndConcurrentOperationsTest.java +++ b/core/src/test/java/org/infinispan/eviction/impl/EvictionWithPassivationAndConcurrentOperationsTest.java @@ -128,8 +128,7 @@ public void testEvictionDuringWriteWithConcurrentRead() throws TimeoutException, // #1 evicted-key evicted by other-key from write // #2 other-key evicted by evicted-key from the get assertEquals(2L, extractComponent(cache, PassivationManager.class).getPassivations()); - // #1 evicted key activated from the get - assertEquals(1L, extractComponent(cache, ActivationManager.class).getActivationCount()); + assertEquals(0L, extractComponent(cache, ActivationManager.class).getActivationCount()); assertEquals(0L, extractComponent(cache, ActivationManager.class).getPendingActivationCount()); } diff --git a/core/src/test/java/org/infinispan/eviction/impl/EvictionWithPassivationTest.java b/core/src/test/java/org/infinispan/eviction/impl/EvictionWithPassivationTest.java index b634a03a6316..3f0aa656388f 100644 --- a/core/src/test/java/org/infinispan/eviction/impl/EvictionWithPassivationTest.java +++ b/core/src/test/java/org/infinispan/eviction/impl/EvictionWithPassivationTest.java @@ -272,7 +272,7 @@ public void testComputeOnEvictedEntry() throws Exception { String evictedKey = evictionListener.getEvictedKey(); assertEntryInStore(evictedKey, true); testCache.compute(evictedKey, (k ,v) -> v + "-modfied"); - assertEntryInStore(evictedKey, false); + assertEntryInStore(evictedKey, true); } public void testRemoveViaComputeOnEvictedEntry() throws Exception { @@ -285,9 +285,6 @@ public void testRemoveViaComputeOnEvictedEntry() throws Exception { // Eviction notification can be non blocking async in certain configs - so wait for notification to complete evictionListener.phaser.awaitAdvanceInterruptibly(phase, 10, TimeUnit.SECONDS); String evictedKey = evictionListener.getEvictedKey(); - if (evictedKey == null) { - System.currentTimeMillis(); - } assertEntryInStore(evictedKey, true); testCache.compute(evictedKey, (k ,v) -> null); assertFalse(testCache.containsKey(evictedKey)); @@ -299,7 +296,7 @@ public void testCleanStoreOnPut() throws Exception { testCache.clear(); putIntoStore("key", "oldValue"); testCache.put("key", "value"); - assertEntryInStore("key", false); + assertEntryInStore("key", true); } public void testConcurrentWriteWithEviction() throws InterruptedException, TimeoutException, ExecutionException { diff --git a/core/src/test/java/org/infinispan/expiration/impl/ExpirationFileStoreDistListenerFunctionalTest.java b/core/src/test/java/org/infinispan/expiration/impl/ExpirationFileStoreDistListenerFunctionalTest.java index ec7a9e3b8a2a..f1b5dd35686c 100644 --- a/core/src/test/java/org/infinispan/expiration/impl/ExpirationFileStoreDistListenerFunctionalTest.java +++ b/core/src/test/java/org/infinispan/expiration/impl/ExpirationFileStoreDistListenerFunctionalTest.java @@ -69,12 +69,6 @@ protected void configure(ConfigurationBuilder config) { } } - @AfterClass(alwaysRun = true) - protected void clearTempDir() { - Util.recursiveFileRemove(PERSISTENT_LOCATION); - Util.recursiveFileRemove(EXTRA_MANAGER_LOCATION); - } - protected void removeFromContainer(String key) { super.removeFromContainer(key); extraCache.getAdvancedCache().getDataContainer().remove(key); @@ -96,6 +90,21 @@ protected void clearContent() { TestingUtil.clearContent(extraManager); } + @Override + protected void teardown() { + super.teardown(); + TestingUtil.killCacheManagers(extraManager); + } + + @AfterClass(alwaysRun = true) + @Override + protected void destroyAfterClass() { + super.destroyAfterClass(); + // Delete the directories after killing all managers + Util.recursiveFileRemove(PERSISTENT_LOCATION); + Util.recursiveFileRemove(EXTRA_MANAGER_LOCATION); + } + @Override protected EmbeddedCacheManager createCacheManager(ConfigurationBuilder builder) { GlobalConfigurationBuilder globalBuilder = GlobalConfigurationBuilder.defaultClusteredBuilder(); diff --git a/core/src/test/java/org/infinispan/jmx/ActivationAndPassivationInterceptorMBeanTest.java b/core/src/test/java/org/infinispan/jmx/ActivationAndPassivationInterceptorMBeanTest.java index 654c19d51bbf..4a99b48d4a7d 100644 --- a/core/src/test/java/org/infinispan/jmx/ActivationAndPassivationInterceptorMBeanTest.java +++ b/core/src/test/java/org/infinispan/jmx/ActivationAndPassivationInterceptorMBeanTest.java @@ -4,7 +4,6 @@ import static org.infinispan.test.TestingUtil.k; import static org.infinispan.test.TestingUtil.v; import static org.testng.AssertJUnit.assertEquals; -import static org.testng.AssertJUnit.assertFalse; import static org.testng.AssertJUnit.assertNotNull; import static org.testng.AssertJUnit.assertNull; import static org.testng.AssertJUnit.assertTrue; @@ -100,8 +99,8 @@ public void testActivationOnGet(Method m) { loader.write(MarshalledEntryUtil.create(k(m), v(m), cache)); assert loader.contains(k(m)); assert cache.get(k(m)).equals(v(m)); - assertActivationCount(1); - assert !loader.contains(k(m)); + assertActivationCount(0); + assert loader.contains(k(m)); } public void testActivationOnPut(Method m) { @@ -112,8 +111,8 @@ public void testActivationOnPut(Method m) { assert loader.contains(k(m)); cache.put(k(m), v(m, 2)); assert cache.get(k(m)).equals(v(m, 2)); - assertActivationCount(1); - assert !loader.contains(k(m)) : "this should only be persisted on evict"; + assertActivationCount(0); + assert loader.contains(k(m)); } public void testActivationOnReplace(Method m) { @@ -126,8 +125,8 @@ public void testActivationOnReplace(Method m) { Object prev = cache.replace(k(m), v(m, 2)); assertNotNull(prev); assertEquals(v(m), prev); - assertActivationCount(1); - assertFalse(loader.contains(k(m))); + assertActivationCount(0); + assertTrue(loader.contains(k(m))); } public void testActivationOnPutMap(Method m) { @@ -140,11 +139,11 @@ public void testActivationOnPutMap(Method m) { Map toAdd = new HashMap<>(); toAdd.put(k(m), v(m, 2)); cache.putAll(toAdd); - assertActivationCount(1); + assertActivationCount(0); Object obj = cache.get(k(m)); assertNotNull(obj); assertEquals(v(m, 2), obj); - assertFalse(loader.contains(k(m))); + assertTrue(loader.contains(k(m))); } public void testPassivationOnEvict(Method m) throws Exception { diff --git a/core/src/test/java/org/infinispan/jmx/CacheMgmtInterceptorMBeanTest.java b/core/src/test/java/org/infinispan/jmx/CacheMgmtInterceptorMBeanTest.java index 0de9a3934a82..a64e340fe3d0 100644 --- a/core/src/test/java/org/infinispan/jmx/CacheMgmtInterceptorMBeanTest.java +++ b/core/src/test/java/org/infinispan/jmx/CacheMgmtInterceptorMBeanTest.java @@ -150,7 +150,7 @@ public void testStores() throws Exception { assertCurrentNumberOfEntriesInMemory(1); assertApproximateEntriesInMemory(1); assertCurrentNumberOfEntries(2); - assertApproximateEntries(2); + assertApproximateEntries(3); resetStats(); diff --git a/core/src/test/java/org/infinispan/persistence/ClusteredConditionalCommandTest.java b/core/src/test/java/org/infinispan/persistence/ClusteredConditionalCommandTest.java index 9e107b9c4b14..903fd4d7ba6d 100644 --- a/core/src/test/java/org/infinispan/persistence/ClusteredConditionalCommandTest.java +++ b/core/src/test/java/org/infinispan/persistence/ClusteredConditionalCommandTest.java @@ -30,7 +30,6 @@ import org.infinispan.test.TestingUtil; import org.infinispan.test.fwk.InCacheMode; import org.infinispan.util.concurrent.IsolationLevel; -import org.testng.SkipException; import org.testng.annotations.AfterMethod; import org.testng.annotations.Test; @@ -108,7 +107,7 @@ private CacheHelper create(List> cacheList) { assertTrue(cacheHelper.addCache(Ownership.BACKUP, cache)); } else { log.debug("Cache " + address(cache) + " is the non owner"); - assertTrue(cacheHelper.addCache(Ownership.NON_OWNER, cache) || cacheMode.isScattered()); + assertTrue(cacheHelper.addCache(Ownership.NON_OWNER, cache)); } } return cacheHelper; @@ -122,10 +121,6 @@ private void doTest(String cacheName, ConditionalOperation operation, Ownership } List> cacheList = getCaches(cacheName); - // These are not valid test combinations - so just ignore them - if (shared && passivation) { - throw new SkipException("Shared passivation is not supported"); - } waitForNoRebalance(cacheList); final CacheHelper cacheHelper = create(cacheList); final boolean skipLoad = flag == Flag.SKIP_CACHE_LOAD || flag == Flag.SKIP_CACHE_STORE; diff --git a/core/src/test/java/org/infinispan/persistence/PassivatePersistentTest.java b/core/src/test/java/org/infinispan/persistence/PassivatePersistentTest.java index 3e07c54fbf14..8d3887ce1a52 100644 --- a/core/src/test/java/org/infinispan/persistence/PassivatePersistentTest.java +++ b/core/src/test/java/org/infinispan/persistence/PassivatePersistentTest.java @@ -2,6 +2,9 @@ import javax.transaction.TransactionManager; +import static org.testng.AssertJUnit.assertEquals; +import static org.testng.AssertJUnit.assertTrue; + import org.infinispan.Cache; import org.infinispan.configuration.cache.ConfigurationBuilder; import org.infinispan.manager.CacheContainer; @@ -47,20 +50,22 @@ public void tearDown() throws PersistenceException { public void testPersistence() throws PersistenceException { cache.put("k", "v"); - assert "v".equals(cache.get("k")); + assertEquals("v", cache.get("k")); cache.evict("k"); - assert store.contains("k"); + assertTrue(store.contains("k")); + + assertEquals("v", cache.get("k")); + + cache.put("k", "v2"); - assert "v".equals(cache.get("k")); - eventually(() -> !store.contains("k")); + assertEquals("v", store.loadEntry("k").getValue()); cache.stop(); cache.start(); // The old store's marshaller is not working any more store = TestingUtil.getFirstStore(cache); - assert store.contains("k"); - assert "v".equals(cache.get("k")); - eventually(() -> !store.contains("k")); + assertEquals("v2", store.loadEntry("k").getValue()); + assertEquals("v2", cache.get("k")); } } diff --git a/core/src/test/java/org/infinispan/persistence/PassivationFunctionalTest.java b/core/src/test/java/org/infinispan/persistence/PassivationFunctionalTest.java index 114349f59b52..5ee5d968f718 100644 --- a/core/src/test/java/org/infinispan/persistence/PassivationFunctionalTest.java +++ b/core/src/test/java/org/infinispan/persistence/PassivationFunctionalTest.java @@ -2,6 +2,7 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.testng.AssertJUnit.assertEquals; +import static org.testng.AssertJUnit.assertNotNull; import java.util.HashMap; import java.util.Map; @@ -64,6 +65,22 @@ public void afterMethod() throws PersistenceException { if (store != null) store.clear(); } + private void assertInCacheAndInStore(Object key, Object cacheValue, long lifespanMillis) throws PersistenceException { + assertInCacheAndInStore(key, cacheValue, cacheValue, lifespanMillis); + } + + private void assertInCacheAndInStore(Object key, Object cacheValue, Object storeValue) throws PersistenceException { + assertInCacheAndInStore(key, cacheValue, storeValue, -1); + } + + private void assertInCacheAndInStore(Object key, Object cacheValue, Object storeValue, long lifespanMillis) throws PersistenceException { + InternalCacheValue se = cache.getAdvancedCache().getDataContainer().get(key).toInternalCacheValue(); + testStoredEntry(se, cacheValue, lifespanMillis, "Cache", key); + MarshallableEntry storeEntry = store.loadEntry(key); + assertNotNull(storeEntry); + assertEquals(storeValue, storeEntry.getValue()); + } + private void assertInCacheNotInStore(Object key, Object value) throws PersistenceException { assertInCacheNotInStore(key, value, -1); } @@ -125,8 +142,8 @@ public void testPassivate() throws PersistenceException { assert cache.get("k1").equals("v1"); assert cache.get("k2").equals("v2"); - assertInCacheNotInStore("k1", "v1"); - assertInCacheNotInStore("k2", "v2", lifespan); + assertInCacheAndInStore("k1", "v1", "v1"); + assertInCacheAndInStore("k2", "v2", lifespan); cache.evict("k1"); cache.evict("k2"); @@ -154,30 +171,30 @@ public void testRemoveAndReplace() throws PersistenceException { assertNotInCacheAndStore("k1"); assert cache.put("k2", "v2-NEW").equals("v2"); - assertInCacheNotInStore("k2", "v2-NEW"); + assertInCacheAndInStore("k2", "v2-NEW", "v2"); cache.evict("k2"); assertInStoreNotInCache("k2", "v2-NEW"); assert cache.replace("k2", "v2-REPLACED").equals("v2-NEW"); - assertInCacheNotInStore("k2", "v2-REPLACED"); + assertInCacheAndInStore("k2", "v2-REPLACED", "v2-NEW"); cache.evict("k2"); assertInStoreNotInCache("k2", "v2-REPLACED"); assert !cache.replace("k2", "some-rubbish", "v2-SHOULDNT-STORE"); // but should activate - assertInCacheNotInStore("k2", "v2-REPLACED"); + assertInCacheAndInStore("k2", "v2-REPLACED", "v2-REPLACED"); cache.evict("k2"); assertInStoreNotInCache("k2", "v2-REPLACED"); assert cache.replace("k2", "v2-REPLACED", "v2-REPLACED-AGAIN"); - assertInCacheNotInStore("k2", "v2-REPLACED-AGAIN"); + assertInCacheAndInStore("k2", "v2-REPLACED-AGAIN", "v2-REPLACED"); cache.evict("k2"); assertInStoreNotInCache("k2", "v2-REPLACED-AGAIN"); assert cache.putIfAbsent("k2", "should-not-appear").equals("v2-REPLACED-AGAIN"); - assertInCacheNotInStore("k2", "v2-REPLACED-AGAIN"); + assertInCacheAndInStore("k2", "v2-REPLACED-AGAIN", "v2-REPLACED-AGAIN"); assert cache.putIfAbsent("k1", "v1-if-absent") == null; assertInCacheNotInStore("k1", "v1-if-absent"); @@ -253,7 +270,7 @@ public void testPutMap() throws PersistenceException { cache.putAll(m); assertInCacheNotInStore("k1", "v1-NEW"); - assertInCacheNotInStore("k2", "v2-NEW"); + assertInCacheAndInStore("k2", "v2-NEW", "v2"); assertInCacheNotInStore("k3", "v3-NEW"); } diff --git a/core/src/test/java/org/infinispan/persistence/manager/PersistenceManagerTest.java b/core/src/test/java/org/infinispan/persistence/manager/PersistenceManagerTest.java index 592e539abaea..3ec7cd853c3e 100644 --- a/core/src/test/java/org/infinispan/persistence/manager/PersistenceManagerTest.java +++ b/core/src/test/java/org/infinispan/persistence/manager/PersistenceManagerTest.java @@ -30,6 +30,7 @@ import org.infinispan.test.TestException; import org.infinispan.test.fwk.CleanupAfterMethod; import org.infinispan.test.fwk.TestCacheManagerFactory; +import org.testng.annotations.Factory; import org.testng.annotations.Test; import io.reactivex.rxjava3.core.Flowable; @@ -44,6 +45,24 @@ @Test(groups = "unit", testName = "persistence.PersistenceManagerTest") @CleanupAfterMethod public class PersistenceManagerTest extends SingleCacheManagerTest { + private boolean transactional; + + private PersistenceManagerTest transactional(boolean transactional) { + this.transactional = transactional; + return this; + } + @Factory + public static Object[] factory() { + return new Object[] { + new PersistenceManagerTest().transactional(true), + new PersistenceManagerTest().transactional(false) + }; + } + + @Override + protected String parameters() { + return "transactional=" + transactional; + } /** * Simulates cache receiving a topology update while stopping. @@ -152,9 +171,20 @@ public void testStoreExceptionInWrite() { assertFalse(store1.contains(key)); } + public void testStoreNotWrittenOnRemoveMiss() { + if (transactional) { + // Only supported for non tx + return; + } + FailStore store2 = getStore(cache, 1, true); + store2.failModification(1); + + cache.remove("k"); + } + @Override protected EmbeddedCacheManager createCacheManager() { - ConfigurationBuilder cfg = getDefaultStandaloneCacheConfig(true); + ConfigurationBuilder cfg = getDefaultStandaloneCacheConfig(transactional); cfg.persistence().addStore(DelayStore.ConfigurationBuilder.class); cfg.persistence().addStore(FailStore.ConfigurationBuilder.class); cfg.persistence().addStore(FailStore.ConfigurationBuilder.class); diff --git a/core/src/test/java/org/infinispan/persistence/sifs/SoftIndexFileStoreFileStatsTest.java b/core/src/test/java/org/infinispan/persistence/sifs/SoftIndexFileStoreFileStatsTest.java index 04279a9c628b..ba2836292d90 100644 --- a/core/src/test/java/org/infinispan/persistence/sifs/SoftIndexFileStoreFileStatsTest.java +++ b/core/src/test/java/org/infinispan/persistence/sifs/SoftIndexFileStoreFileStatsTest.java @@ -47,7 +47,9 @@ protected void setUpTempDir() { } @AfterClass(alwaysRun = true) - protected void clearTempDir() { + @Override + protected void destroyAfterClass() { + super.destroyAfterClass(); Util.recursiveFileRemove(tmpDirectory); } diff --git a/core/src/test/java/org/infinispan/persistence/sifs/SoftIndexFileStoreFunctionalTest.java b/core/src/test/java/org/infinispan/persistence/sifs/SoftIndexFileStoreFunctionalTest.java index 6d5064bdba09..1d081b56965c 100644 --- a/core/src/test/java/org/infinispan/persistence/sifs/SoftIndexFileStoreFunctionalTest.java +++ b/core/src/test/java/org/infinispan/persistence/sifs/SoftIndexFileStoreFunctionalTest.java @@ -1,22 +1,55 @@ package org.infinispan.persistence.sifs; +import static org.testng.AssertJUnit.assertEquals; +import static org.testng.AssertJUnit.fail; + import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; import org.infinispan.Cache; import org.infinispan.commons.test.CommonsTestingUtil; import org.infinispan.commons.util.Util; import org.infinispan.configuration.cache.ConfigurationBuilder; import org.infinispan.configuration.cache.PersistenceConfigurationBuilder; +import org.infinispan.distribution.ch.KeyPartitioner; import org.infinispan.persistence.BaseStoreFunctionalTest; import org.infinispan.test.TestingUtil; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Factory; import org.testng.annotations.Test; @Test(groups = "unit", testName = "persistence.sifs.SoftIndexFileStoreFunctionalTest") public class SoftIndexFileStoreFunctionalTest extends BaseStoreFunctionalTest { protected String tmpDirectory; + protected int segmentCount; + + public SoftIndexFileStoreFunctionalTest(int segmentCount) { + this.segmentCount = segmentCount; + } + + @Factory + public static Object[] factory() { + return new Object[] { + new SoftIndexFileStoreFunctionalTest(1), + new SoftIndexFileStoreFunctionalTest(10), + new SoftIndexFileStoreFunctionalTest(256), + new SoftIndexFileStoreFunctionalTest(2048), + }; + } + + @Override + protected String parameters() { + return "[" + segmentCount + "]"; + } + @BeforeClass(alwaysRun = true) protected void setUpTempDir() { tmpDirectory = CommonsTestingUtil.tmpDirectory(getClass()); @@ -27,6 +60,13 @@ protected void clearTempDir() { Util.recursiveFileRemove(tmpDirectory); } + @Override + protected ConfigurationBuilder getDefaultCacheConfiguration() { + ConfigurationBuilder configurationBuilder = super.getDefaultCacheConfiguration(); + configurationBuilder.clustering().hash().numSegments(segmentCount); + return configurationBuilder; + } + @Override protected PersistenceConfigurationBuilder createCacheStoreConfig(PersistenceConfigurationBuilder persistence, String cacheName, boolean preload) { @@ -54,4 +94,46 @@ public void testWritingSameKeyShortTimes() { cache.remove("k"); } + + @DataProvider(name = "keyArgs") + public Object[][] keyConfiguration() { + return Stream.of(10, 10_000, 250_000) + .flatMap(keyCount -> + Stream.of(Boolean.TRUE, Boolean.FALSE) + .map(largeKey -> new Object[] { + keyCount, largeKey + }) + ).toArray(Object[][]::new); + } + + @Test(dataProvider = "keyArgs") + public void testWriteManyDifferentKeysAndIterate(int keyCount, boolean largeKey) { + String cacheName = "testWriteManyDifferentKeysAndIterate"; + ConfigurationBuilder cb = getDefaultCacheConfiguration(); + createCacheStoreConfig(cb.persistence(), cacheName, false); + TestingUtil.defineConfiguration(cacheManager, cacheName, cb.build()); + + Cache cache = cacheManager.getCache(cacheName); + + for (int i = 0; i < keyCount; ++i) { + int anotherValue = i * 13 + (i-1) * 19; + String key = "k" + i + (largeKey ? "-" + anotherValue : ""); + cache.put(key, "v" + i + "-" + anotherValue); + } + + // Force to read from store + cache.getAdvancedCache().getDataContainer().clear(); + + var list = new ArrayList<>(cache.entrySet()); + if (list.size() > keyCount) { + Set duplicateKeys = new HashSet<>(); + var dupList = list.stream().map(Map.Entry::getKey) + .filter(k -> !duplicateKeys.add(k)) + .map(k -> Map.entry(k, TestingUtil.extractComponent(cache, KeyPartitioner.class).getSegment(k))) + .collect(Collectors.toList()); + fail("List contained a duplicate element" + dupList); + } else { + assertEquals(keyCount, list.size()); + } + } } diff --git a/core/src/test/java/org/infinispan/persistence/sifs/SoftIndexFileStoreRestartTest.java b/core/src/test/java/org/infinispan/persistence/sifs/SoftIndexFileStoreRestartTest.java index fa10a2eafbb8..a19dc2291e2f 100644 --- a/core/src/test/java/org/infinispan/persistence/sifs/SoftIndexFileStoreRestartTest.java +++ b/core/src/test/java/org/infinispan/persistence/sifs/SoftIndexFileStoreRestartTest.java @@ -28,7 +28,6 @@ import org.infinispan.test.Mocks; import org.infinispan.test.TestingUtil; import org.infinispan.test.fwk.CheckPoint; -import org.infinispan.util.concurrent.CompletionStages; import org.testng.annotations.AfterClass; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeClass; @@ -247,12 +246,15 @@ public void testRestartWithEntryUpdatedMultipleTimes(boolean leafOrNode) throws createCacheManagers(); + assertEquals("value-" + (size - 1), cache(0, cacheName).get(key)); + WaitDelegatingNonBlockingStore store = TestingUtil.getFirstStoreWait(cache(0, cacheName)); Compactor compactor = TestingUtil.extractField(store.delegate(), "compactor"); // Force compaction for the previous file - CompletionStages.join(compactor.forceCompactionForAllNonLogFiles()); + compactor.forceCompactionForAllNonLogFiles() + .toCompletableFuture().get(10, TimeUnit.SECONDS); assertEquals("value-" + (size - 1), cache(0, cacheName).get(key)); @@ -303,6 +305,7 @@ public void testRestartCompactorNotComplete() throws Throwable { ConcurrentMap stats; // Insert until compactor has filled a file - which we can force compaction on below while ((stats = compactor.getFileStats()).isEmpty()) { + cache(0, cacheName).remove("key-" + size); cache(0, cacheName).put("key-" + size, "value-" + size); size++; } diff --git a/core/src/test/java/org/infinispan/persistence/sifs/SoftIndexFileStoreSplitInnerNodeTest.java b/core/src/test/java/org/infinispan/persistence/sifs/SoftIndexFileStoreSplitInnerNodeTest.java index b8748af39e22..b83513fab5d5 100644 --- a/core/src/test/java/org/infinispan/persistence/sifs/SoftIndexFileStoreSplitInnerNodeTest.java +++ b/core/src/test/java/org/infinispan/persistence/sifs/SoftIndexFileStoreSplitInnerNodeTest.java @@ -34,12 +34,12 @@ protected void clearTempDir() { @Override protected void createCacheManagers() { ConfigurationBuilder cb = getDefaultClusteredCacheConfig(CacheMode.DIST_SYNC, false); + cb.clustering().hash().numSegments(1); cb.memory().maxCount((long) Math.ceil(MAX_ENTRIES * 0.2)) .persistence().passivation(false) .addSoftIndexFileStore() .dataLocation(Paths.get(tmpDirectory, "data").toString()) .indexLocation(Paths.get(tmpDirectory, "index").toString()) - .indexSegments(1) .purgeOnStartup(true) .preload(false) .expiration().wakeUpInterval(Long.MAX_VALUE) diff --git a/core/src/test/java/org/infinispan/persistence/sifs/SoftIndexFileStoreTest.java b/core/src/test/java/org/infinispan/persistence/sifs/SoftIndexFileStoreTest.java index 153c18a62b60..bd89f133011d 100644 --- a/core/src/test/java/org/infinispan/persistence/sifs/SoftIndexFileStoreTest.java +++ b/core/src/test/java/org/infinispan/persistence/sifs/SoftIndexFileStoreTest.java @@ -13,6 +13,7 @@ import java.util.ArrayDeque; import java.util.Queue; import java.util.concurrent.CompletionStage; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; @@ -22,6 +23,7 @@ import org.infinispan.commons.test.CommonsTestingUtil; import org.infinispan.commons.test.Exceptions; +import org.infinispan.commons.util.IntSets; import org.infinispan.commons.util.Util; import org.infinispan.configuration.cache.Configuration; import org.infinispan.configuration.cache.ConfigurationBuilder; @@ -62,17 +64,17 @@ protected NonBlockingStore createStore() { @Override protected Configuration buildConfig(ConfigurationBuilder configurationBuilder) { + configurationBuilder.clustering().hash().numSegments(2); return configurationBuilder.persistence() .addSoftIndexFileStore() .dataLocation(Paths.get(tmpDirectory, "data").toString()) .indexLocation(Paths.get(tmpDirectory, "index").toString()) - .indexSegments(1) .maxFileSize(1000) .build(); } // test for ISPN-5753 - public void testOverrideWithExpirableAndCompaction() throws InterruptedException { + public void testOverrideWithExpirableAndCompaction() { // write immortal entry store.write(marshalledEntry(internalCacheEntry("key", "value1", -1))); writeGibberish(-1, true); // make sure that compaction happens - value1 is compacted @@ -113,13 +115,15 @@ public void testStopWithCompactorIndexNotComplete() throws InterruptedException, Index index = TestingUtil.extractField(compactor, "index"); FlowableProcessor[] processors = TestingUtil.extractField(index, "flowableProcessors"); - assertEquals(1, processors.length); + assertEquals(2, processors.length); FlowableProcessor original = processors[0]; Queue queue = new ArrayDeque<>(); - processors[0] = UnicastProcessor.create(); - processors[0].serialize().subscribe(queue::add); + UnicastProcessor unicastProcessor = UnicastProcessor.create(); + unicastProcessor.serialize().subscribe(queue::add); + + processors[0] = unicastProcessor; CountDownLatch latch = new CountDownLatch(1); @@ -265,4 +269,107 @@ public void onError(Throwable t) { // Only a single entry was expired. assertThat(expired.get()).isEqualTo(1); } + + public void testRemoveSegmentsCleansUpProperly() throws ExecutionException, InterruptedException, TimeoutException { + Compactor compactor = TestingUtil.extractField(store.delegate(), "compactor"); + var fileStats = compactor.getFileStats(); + assertEquals(0, fileStats.size()); + // We force the entry into segment 0 + TestingUtil.join(store.write(0, marshalledEntry(internalCacheEntry("foo", "bar", 10)))); + + // Now remove the segment + TestingUtil.join(store.removeSegments(IntSets.immutableSet(0))); + + assertNull(TestingUtil.join(store.load(0, "foo"))); + + verifyStatsHaveNoData(-77, fileStats); + + // Add the segment back... the value should be gone still + TestingUtil.join(store.addSegments(IntSets.immutableSet(0))); + + assertNull(TestingUtil.join(store.load(0, "foo"))); + + // Note this is -77 since we don't set the size of the file until we complete it or shutdown + verifyStatsHaveNoData(-77, fileStats); + + // Stop the store so we can restart to test if the entry is still gone or not + store.stopAndWait(); + + // Restart to prevent other test failures + startStore(store); + + // Technically this will fail if the index is deleted... however that is not an issue as this test is + // really for DIST Cache mode, and the store should ALWAYS have purgeOnStartup enabled which would prevent this from + // being an issue + assertNull(TestingUtil.join(store.load(0, "foo"))); + + compactor = TestingUtil.extractField(store.delegate(), "compactor"); + + // When the store restarted it compacted the file away + fileStats = compactor.getFileStats(); + assertTrue("fileStats were: " + fileStats, fileStats.isEmpty()); + + assertEquals(0, SoftIndexFileStoreTestUtils.dataDirectorySize(tmpDirectory, "mock-cache")); + } + + private void verifyStatsHaveNoData(long expected, ConcurrentMap fileStats) { + long sizeAfterAddingBack = 0; + // Note stats file still may be empty here + for (Compactor.Stats stats : fileStats.values()) { + sizeAfterAddingBack -= stats.getFree(); + if (stats.getTotal() > 0) { + sizeAfterAddingBack += stats.getTotal(); + } + } + + assertEquals(expected, sizeAfterAddingBack); + } + + public void testFileStatsWriteNotOwnedSegment() throws ExecutionException, InterruptedException, TimeoutException { + Compactor compactor = TestingUtil.extractField(store.delegate(), "compactor"); + var fileStats = compactor.getFileStats(); + assertEquals(0, fileStats.size()); + + TestingUtil.join(store.write(0, marshalledEntry(internalCacheEntry("foo-0", "bar-0", 10)))); + + assertTrue(fileStats.isEmpty()); + + TestingUtil.join(store.removeSegments(IntSets.immutableSet(1))); + + TestingUtil.join(store.write(1, marshalledEntry(internalCacheEntry("foo-1", "bar-1", 10)))); + + // This should contain free data since we wrote an entry to a segment that we no longer own + verifyStatsHaveNoData(-81, fileStats); + } + + public void testFileStatsAfterRemovingSegment() throws ExecutionException, InterruptedException, TimeoutException { + Compactor compactor = TestingUtil.extractField(store.delegate(), "compactor"); + var fileStats = compactor.getFileStats(); + assertEquals(0, fileStats.size()); + + TestingUtil.join(store.write(1, marshalledEntry(internalCacheEntry("foo-1", "bar-1", 10)))); + + TestingUtil.join(store.removeSegments(IntSets.immutableSet(1))); + + // This should contain free data since we wrote an entry to a segment that we no longer own + verifyStatsHaveNoData(-81, fileStats); + } + + public void testFileStatsAfterRemovingWithRemovedEntry() throws ExecutionException, InterruptedException, TimeoutException { + Compactor compactor = TestingUtil.extractField(store.delegate(), "compactor"); + var fileStats = compactor.getFileStats(); + assertEquals(0, fileStats.size()); + + TestingUtil.join(store.write(1, marshalledEntry(internalCacheEntry("foo-1", "bar-1", 10)))); + + TestingUtil.join(store.delete(1, "foo-1")); + + // Removed entry information doesn't currently count towards free space see ISPN-15246 + verifyStatsHaveNoData(-81, fileStats); + + TestingUtil.join(store.removeSegments(IntSets.immutableSet(1))); + + // After removing the segment even the removed entries are updated in stats + verifyStatsHaveNoData(-123, fileStats); + } } diff --git a/core/src/test/java/org/infinispan/test/TestingUtil.java b/core/src/test/java/org/infinispan/test/TestingUtil.java index 2b6fde094be7..0aae9830f874 100644 --- a/core/src/test/java/org/infinispan/test/TestingUtil.java +++ b/core/src/test/java/org/infinispan/test/TestingUtil.java @@ -33,6 +33,7 @@ import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; +import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; @@ -204,6 +205,10 @@ public static CompletionStage sequenceAsync(CompletionStage first, Cal return first.thenComposeAsync(ignored -> Exceptions.unchecked(second), executor); } + public static T join(CompletionStage stage) throws ExecutionException, InterruptedException, java.util.concurrent.TimeoutException { + return stage.toCompletableFuture().get(10, TimeUnit.SECONDS); + } + /** * Should be used by tests for a timeout when they need to wait for that timeout to expire. * diff --git a/documentation/src/main/asciidoc/topics/con_file_based_cache_stores.adoc b/documentation/src/main/asciidoc/topics/con_file_based_cache_stores.adoc index 3515ef91ac68..83f4bf2c04fc 100644 --- a/documentation/src/main/asciidoc/topics/con_file_based_cache_stores.adoc +++ b/documentation/src/main/asciidoc/topics/con_file_based_cache_stores.adoc @@ -21,6 +21,11 @@ When append-only files: * Reach their maximum size, {brandname} creates a new file and starts writing to it. * Reach the compaction threshold of less than 50% usage, {brandname} overwrites the entries to a new file and then deletes the old file. +[NOTE] +==== +SoftIndexFileStore should use purge on startup to ensure stale entries are not resurrected. +==== + .B+ trees To improve performance, append-only files in a `SoftIndexFileStore` are indexed using a **B+ Tree** that can be stored both on disk and in memory. @@ -29,23 +34,31 @@ The in-memory index uses Java soft references to ensure it can be rebuilt if rem Because `SoftIndexFileStore` uses Java soft references to keep indexes in memory, it helps prevent out-of-memory exceptions. GC removes indexes before they consume too much memory while still falling back to disk. -You can configure any number of B+ trees with the `segments` attribute on the `index` element declaratively or with the `indexSegments()` method programmatically. -By default {brandname} creates up to 16 B+ trees, which means there can be up to 16 indexes. -Having multiple indexes prevents bottlenecks from concurrent writes to an index and reduces the number of entries that {brandname} needs to keep in memory. -As it iterates over a soft-index file store, {brandname} reads all entries in an index at the same time. +`SoftIndexFileStore` creates a B+ tree per configured cache segment. +This provides an additional "index" as it only has so many elements and provides additional parallelism for index updates. +Currently we allow for a parallel amount based on one sixteenth of the number of cache segments. Each entry in the B+ tree is a node. By default, the size of each node is limited to 4096 bytes. `SoftIndexFileStore` throws an exception if keys are longer after serialization occurs. +.File limits + +`SoftIndexFileStore` will use two plus the configured openFilesLimit amount of files at a given time. +The two additional file pointers are reserved for the log appender for newly updated data and another +for the compactor which writes compacted entries into a new file. + +The amount of open allocated files allocated for indexing is one tenth of the total number of the configured openFilesLimit. +This number has a minimum of 1 or the number of cache segments. +Any number remaning from configured limit is allocated for open data files themselves. + .Segmentation -Soft-index file stores are always segmented. +Soft-index file stores are always segmented. The append log(s) are not directly segmented and segmentation is handled directly by the index. -[NOTE] -==== -The `AdvancedStore.purgeExpired()` method is not implemented in `SoftIndexFileStore`. -==== +.Expiration + +The SoftIndexFileStore has full support for expired entries and their requirements. [discrete] == Single File Cache Stores diff --git a/documentation/src/main/asciidoc/topics/con_how_passivation_works.adoc b/documentation/src/main/asciidoc/topics/con_how_passivation_works.adoc index 1f50e58a1ba8..5ecc3de1f5ff 100644 --- a/documentation/src/main/asciidoc/topics/con_how_passivation_works.adoc +++ b/documentation/src/main/asciidoc/topics/con_how_passivation_works.adoc @@ -7,54 +7,51 @@ Writes to data in memory result in writes to persistent storage. If {brandname} evicts data from memory, then data in persistent storage includes entries that are evicted from memory. In this way persistent storage is a superset of the in-memory cache. +This is recommended when you require highest consistency as the store will be able to be read again after a crash. If you do not configure eviction, then data in persistent storage provides a copy of data in memory. .Passivation enabled {brandname} adds data to persistent storage only when it evicts data from -memory. +memory, an entry is removed or upon shutting down the node. -When {brandname} activates entries, it restores data in memory and deletes data -from persistent storage. In this way, data in memory and data in persistent -storage form separate subsets of the entire data set, with no intersection -between the two. +When {brandname} activates entries, it restores data in memory but keeps the data in the store still. +This allows for writes to be just as fast as without a store, and still maintains consistency. +When an entry is created or updated only the in memory will be updated and thus +the store will be outdated for the time being. [NOTE] ==== -Entries in persistent storage can become stale when using shared cache stores. This occurs because {brandname} does not delete passivated entries from shared cache stores when they are activated. - -Values are updated in memory but previously passivated entries remain in persistent storage with out of date values. +Passivation is not supported when a store is also configured as shared. +This is due to entries can become out of sync between nodes depending on when a write is evicted versus read. ==== +To gurantee data consistency any store that is not shared should always have `purgeOnStartup` enabled. +This is true for both passivation enabled or disabled since a store could hold an outdated entry while down and resurrect it at a later point. + The following table shows data in memory and in persistent storage after a series of operations: [options="header"] |=============== -|Operation|Passivation disabled|Passivation enabled|Passivation enabled with shared cache store +|Operation|Passivation disabled|Passivation enabled |Insert k1.|*Memory:* k1 + *Disk:* k1|*Memory:* k1 + -*Disk:* -|*Memory:* k1 + *Disk:* - |Insert k2.|*Memory:* k1, k2 + *Disk:* k1, k2|*Memory:* k1, k2 + -*Disk:* -|*Memory:* k1, k2 + *Disk:* - |Eviction thread runs and evicts k1.|*Memory:* k2 + *Disk:* k1, k2|*Memory:* k2 + -*Disk:* k1|*Memory:* k2 + *Disk:* k1 |Read k1.|*Memory:* k1, k2 + *Disk:* k1, k2|*Memory:* k1, k2 + -*Disk:* -|*Memory:* k1, k2 + *Disk:* k1 |Eviction thread runs and evicts k2.|*Memory:* k1 + *Disk:* k1, k2|*Memory:* k1 + -*Disk:* k2|*Memory:* k1 + *Disk:* k1, k2 |Remove k2.|*Memory:* k1 + *Disk:* k1|*Memory:* k1 + -*Disk:* -|*Memory:* k1 + *Disk:* k1 |=============== diff --git a/documentation/src/main/asciidoc/topics/con_passivation.adoc b/documentation/src/main/asciidoc/topics/con_passivation.adoc index 7687cd4809b3..8c1b1dabd3ac 100644 --- a/documentation/src/main/asciidoc/topics/con_passivation.adoc +++ b/documentation/src/main/asciidoc/topics/con_passivation.adoc @@ -1,10 +1,8 @@ [id='passivation_{context}'] = Passivation Passivation configures {brandname} to write entries to cache stores when it -evicts those entries from memory. In this way, passivation ensures that only a -single copy of an entry is maintained, either in-memory or in a cache store, -which prevents unnecessary and potentially expensive writes to persistent -storage. +evicts those entries from memory. In this way, passivation prevents unnecessary +and potentially expensive writes to persistent storage. Activation is the process of restoring entries to memory from the cache store when there is an attempt to access passivated entries. For this reason, when you @@ -16,7 +14,7 @@ When {brandname} evicts an entry from the cache, it notifies cache listeners that the entry is passivated then stores the entry in the cache store. When {brandname} gets an access request for an evicted entry, it lazily loads the entry from the cache store into memory and then notifies cache listeners that -the entry is activated. +the entry is activated while keeping the value still in the store. [NOTE] ==== diff --git a/documentation/src/main/asciidoc/topics/con_shared_cache_stores.adoc b/documentation/src/main/asciidoc/topics/con_shared_cache_stores.adoc index 92a4ad1b66f2..f1646a89ad08 100644 --- a/documentation/src/main/asciidoc/topics/con_shared_cache_stores.adoc +++ b/documentation/src/main/asciidoc/topics/con_shared_cache_stores.adoc @@ -6,7 +6,7 @@ By default, cache stores are local (`shared="false"`). * Local cache stores are unique to each node; for example, a file-based cache store that persists data to the host filesystem. + -Local cache stores can use "purge on startup" to avoid loading stale entries from persistent storage. +Local cache stores should use "purge on startup" to avoid loading stale entries from persistent storage. * Shared cache stores allow multiple nodes to use the same persistent storage; for example, a JDBC cache store that allows multiple nodes to access the same database. + diff --git a/documentation/src/main/asciidoc/topics/upgrading.adoc b/documentation/src/main/asciidoc/topics/upgrading.adoc index 321d5c44c185..25e0002cf9b1 100644 --- a/documentation/src/main/asciidoc/topics/upgrading.adoc +++ b/documentation/src/main/asciidoc/topics/upgrading.adoc @@ -120,6 +120,10 @@ an unclean shutdown. The behavior can be changed by configuring the global state == Persistence The default `availability-interval` has been increased to 30 seconds. The previous default was 1 second. +=== SoftIndexFileStore + +The index segment configuration is no longer used and instead the number of segments is determined solely by the number of cache segments. + == RESP endpoint The RESP endpoint cache now requires the key storage media type to be diff --git a/hibernate/cache-commons/src/main/java/org/infinispan/hibernate/cache/commons/access/NonTxInvalidationInterceptor.java b/hibernate/cache-commons/src/main/java/org/infinispan/hibernate/cache/commons/access/NonTxInvalidationInterceptor.java index bb319e54459d..213e746b23ac 100644 --- a/hibernate/cache-commons/src/main/java/org/infinispan/hibernate/cache/commons/access/NonTxInvalidationInterceptor.java +++ b/hibernate/cache-commons/src/main/java/org/infinispan/hibernate/cache/commons/access/NonTxInvalidationInterceptor.java @@ -125,6 +125,7 @@ protected Log getLog() { } private Object handleWriteReturn(InvocationContext ctx, RemoveCommand removeCmd, Object rv) { + // Invalidation always has to send even if the command says not to replicate if ( removeCmd.isSuccessful()) { return invalidateAcrossCluster(removeCmd, true, removeCmd.getKey(), removeCmd.getKeyLockOwner()); } @@ -132,6 +133,7 @@ private Object handleWriteReturn(InvocationContext ctx, RemoveCommand removeCmd, } private Object handleEvictReturn(InvocationContext ctx, RemoveCommand removeCmd, Object rv) { + // Invalidation always has to send even if the command says not to replicate if ( removeCmd.isSuccessful()) { return invalidateAcrossCluster(removeCmd, false, removeCmd.getKey(), removeCmd.getKeyLockOwner()); } diff --git a/persistence/jdbc/src/test/java/org/infinispan/persistence/jdbc/stringbased/AbstractStringBasedCacheStore.java b/persistence/jdbc/src/test/java/org/infinispan/persistence/jdbc/stringbased/AbstractStringBasedCacheStore.java index 5d214f5ec889..e20ace9b808e 100644 --- a/persistence/jdbc/src/test/java/org/infinispan/persistence/jdbc/stringbased/AbstractStringBasedCacheStore.java +++ b/persistence/jdbc/src/test/java/org/infinispan/persistence/jdbc/stringbased/AbstractStringBasedCacheStore.java @@ -97,7 +97,7 @@ public void testPutGetRemoveWithPassivationWithoutPreload() throws Exception { cache.get("k1"); cache.get("k2"); assertEquals(2, cache.getAdvancedCache().withFlags(Flag.SKIP_CACHE_LOAD).size()); - eventuallyEquals(1, () -> getAllRows().size()); + eventuallyEquals(2, () -> getAllRows().size()); cache.stop(); cache.start();