From ce4f6292d6350a2c6b82d995352fdf6d07042c9c Mon Sep 17 00:00:00 2001 From: Pedro Ruivo Date: Tue, 12 May 2015 16:37:46 +0100 Subject: [PATCH] ISPN-5046 PartitionHandling: split during commit can leave the cache inconsistent after merge * keep the transaction resources while the partition is not healed. * when the merge happens, complete the pending transactions. --- .../org/infinispan/cache/impl/CacheImpl.java | 8 +- .../TxCompletionNotificationCommand.java | 2 +- .../PartitionHandlingManagerFactory.java | 7 +- .../interceptors/TxInterceptor.java | 186 +++++----- .../BaseDistributionInterceptor.java | 65 +--- .../NonTxDistributionInterceptor.java | 12 +- .../TxDistributionInterceptor.java | 206 ++++++----- .../VersionedDistributionInterceptor.java | 11 +- .../locking/AbstractTxLockingInterceptor.java | 30 +- .../TotalOrderDistributionInterceptor.java | 11 +- ...OrderVersionedDistributionInterceptor.java | 10 +- .../AvailablePartitionHandlingManager.java | 94 +++++ .../impl/PartitionHandlingInterceptor.java | 2 +- .../impl/PartitionHandlingManager.java | 92 ++++- .../impl/PartitionHandlingManagerImpl.java | 312 ++++++++++++++-- .../AbstractDelegatingTransport.java | 4 + .../transport/jgroups/JGroupsTransport.java | 9 + .../statetransfer/StateConsumerImpl.java | 24 +- .../StateTransferInterceptor.java | 20 +- .../statetransfer/StateTransferManager.java | 2 +- .../StateTransferManagerImpl.java | 20 +- .../topology/LocalTopologyManagerImpl.java | 36 +- .../impl/AbstractEnlistmentAdapter.java | 18 +- .../transaction/impl/TransactionTable.java | 44 +-- .../SynchronizationAdapter.java | 5 +- .../transaction/tm/DummyTransaction.java | 10 +- .../transaction/xa/TransactionXaAdapter.java | 15 +- .../transaction/xa/XaTransactionTable.java | 2 +- .../xa/recovery/RecoveryAdminOperations.java | 2 +- .../xa/recovery/RecoveryManager.java | 63 ++-- .../xa/recovery/RecoveryManagerImpl.java | 7 +- .../infinispan/api/ForceWriteLockTest.java | 12 +- .../infinispan/lock/ExplicitUnlockTest.java | 4 +- .../OptimisticTxFailureAfterLockingTest.java | 2 +- .../PessimistTxFailureAfterLockingTest.java | 2 +- .../StaleEagerLocksOnPrepareFailureTest.java | 8 +- .../lock/StaleLocksOnPrepareFailureTest.java | 2 +- .../lock/StaleLocksTransactionTest.java | 4 +- .../AbstractInitiatorCrashTest.java | 8 +- .../MainOwnerChangesPessimisticLockTest.java | 8 +- .../OriginatorBecomesOwnerLockTest.java | 2 +- .../InitiatorCrashPessimisticTest.java | 12 +- .../InitiatorCrashOptimisticReplTest.java | 8 +- .../InitiatorCrashPessimisticReplTest.java | 8 +- ...BaseOptimisticTxPartitionAndMergeTest.java | 79 +++++ .../BasePartitionHandlingTest.java | 41 ++- ...asePessimisticTxPartitionAndMergeTest.java | 83 +++++ .../BaseTxPartitionAndMergeTest.java | 334 ++++++++++++++++++ ...icTxPartitionAndMergeDuringCommitTest.java | 94 +++++ ...cTxPartitionAndMergeDuringPrepareTest.java | 78 ++++ ...TxPartitionAndMergeDuringRollbackTest.java | 105 ++++++ ...cTxPartitionAndMergeDuringPrepareTest.java | 99 ++++++ ...TxPartitionAndMergeDuringRollbackTest.java | 109 ++++++ ...cTxPartitionAndMergeDuringRuntimeTest.java | 76 ++++ .../infinispan/replication/AsyncReplTest.java | 20 +- .../replication/SyncReplLockingTest.java | 8 +- ...ocksWithCommitDuringStateTransferTest.java | 12 +- .../infinispan/test/AbstractCacheTest.java | 28 +- .../test/AbstractInfinispanTest.java | 28 +- .../test/MultipleCacheManagersTest.java | 52 ++- .../test/arquillian/DatagridManager.java | 2 +- .../tx/StaleLockAfterTxAbortTest.java | 4 +- ...TransactionXaAdapterTmIntegrationTest.java | 13 +- .../recovery/PostCommitRecoveryStateTest.java | 4 +- ...seTxClusterExtendedStatisticLogicTest.java | 12 +- 65 files changed, 2116 insertions(+), 574 deletions(-) create mode 100644 core/src/main/java/org/infinispan/partitionhandling/impl/AvailablePartitionHandlingManager.java create mode 100644 core/src/test/java/org/infinispan/partitionhandling/BaseOptimisticTxPartitionAndMergeTest.java create mode 100644 core/src/test/java/org/infinispan/partitionhandling/BasePessimisticTxPartitionAndMergeTest.java create mode 100644 core/src/test/java/org/infinispan/partitionhandling/BaseTxPartitionAndMergeTest.java create mode 100644 core/src/test/java/org/infinispan/partitionhandling/OptimisticTxPartitionAndMergeDuringCommitTest.java create mode 100644 core/src/test/java/org/infinispan/partitionhandling/OptimisticTxPartitionAndMergeDuringPrepareTest.java create mode 100644 core/src/test/java/org/infinispan/partitionhandling/OptimisticTxPartitionAndMergeDuringRollbackTest.java create mode 100644 core/src/test/java/org/infinispan/partitionhandling/PessimisticTxPartitionAndMergeDuringPrepareTest.java create mode 100644 core/src/test/java/org/infinispan/partitionhandling/PessimisticTxPartitionAndMergeDuringRollbackTest.java create mode 100644 core/src/test/java/org/infinispan/partitionhandling/PessimisticTxPartitionAndMergeDuringRuntimeTest.java diff --git a/core/src/main/java/org/infinispan/cache/impl/CacheImpl.java b/core/src/main/java/org/infinispan/cache/impl/CacheImpl.java index 713e467012f..4ba55248501 100644 --- a/core/src/main/java/org/infinispan/cache/impl/CacheImpl.java +++ b/core/src/main/java/org/infinispan/cache/impl/CacheImpl.java @@ -948,11 +948,7 @@ public String getCacheStatus() { @Override public AvailabilityMode getAvailability() { - if (partitionHandlingManager != null) { - return partitionHandlingManager.getAvailabilityMode(); - } else { - return AvailabilityMode.AVAILABLE; - } + return partitionHandlingManager.getAvailabilityMode(); } @Override @@ -1071,7 +1067,7 @@ public Stats getStats() { @Override public XAResource getXAResource() { - return new TransactionXaAdapter(txTable, recoveryManager, txCoordinator, commandsFactory, rpcManager, null, config, name); + return new TransactionXaAdapter(txTable, recoveryManager, txCoordinator, commandsFactory, rpcManager, null, config, name, partitionHandlingManager); } @Override diff --git a/core/src/main/java/org/infinispan/commands/remote/recovery/TxCompletionNotificationCommand.java b/core/src/main/java/org/infinispan/commands/remote/recovery/TxCompletionNotificationCommand.java index 1ad0ecd143c..4916a7e0839 100644 --- a/core/src/main/java/org/infinispan/commands/remote/recovery/TxCompletionNotificationCommand.java +++ b/core/src/main/java/org/infinispan/commands/remote/recovery/TxCompletionNotificationCommand.java @@ -114,7 +114,7 @@ public GlobalTransaction getGlobalTransaction() { private void forwardCommandRemotely(RemoteTransaction remoteTx) { Set affectedKeys = remoteTx.getAffectedKeys(); log.tracef("Invoking forward of TxCompletionNotification for transaction %s. Affected keys: %s", gtx, affectedKeys); - stateTransferManager.forwardCommandIfNeeded(this, affectedKeys, remoteTx.getGlobalTransaction().getAddress(), false); + stateTransferManager.forwardCommandIfNeeded(this, affectedKeys, remoteTx.getGlobalTransaction().getAddress()); } @Override diff --git a/core/src/main/java/org/infinispan/factories/PartitionHandlingManagerFactory.java b/core/src/main/java/org/infinispan/factories/PartitionHandlingManagerFactory.java index 9240002f8b3..689d3b37a62 100644 --- a/core/src/main/java/org/infinispan/factories/PartitionHandlingManagerFactory.java +++ b/core/src/main/java/org/infinispan/factories/PartitionHandlingManagerFactory.java @@ -2,6 +2,7 @@ import org.infinispan.factories.annotations.DefaultFactoryFor; +import org.infinispan.partitionhandling.impl.AvailablePartitionHandlingManager; import org.infinispan.partitionhandling.impl.PartitionHandlingManager; import org.infinispan.partitionhandling.impl.PartitionHandlingManagerImpl; @@ -10,8 +11,8 @@ * @since 7.0 */ @DefaultFactoryFor(classes = PartitionHandlingManager.class) -public class PartitionHandlingManagerFactory extends AbstractNamedCacheComponentFactory implements - AutoInstantiableFactory { +public class PartitionHandlingManagerFactory extends AbstractNamedCacheComponentFactory implements AutoInstantiableFactory { + @Override @SuppressWarnings("unchecked") public T construct(Class componentType) { @@ -21,6 +22,6 @@ public T construct(Class componentType) { return (T) new PartitionHandlingManagerImpl(); } } - return null; + return (T) AvailablePartitionHandlingManager.getInstance(); } } diff --git a/core/src/main/java/org/infinispan/interceptors/TxInterceptor.java b/core/src/main/java/org/infinispan/interceptors/TxInterceptor.java index 304c7229e86..9499138110c 100644 --- a/core/src/main/java/org/infinispan/interceptors/TxInterceptor.java +++ b/core/src/main/java/org/infinispan/interceptors/TxInterceptor.java @@ -30,8 +30,11 @@ import org.infinispan.commands.write.ReplaceCommand; import org.infinispan.commands.write.WriteCommand; import org.infinispan.configuration.cache.Configuration; +import org.infinispan.configuration.cache.Configurations; import org.infinispan.context.Flag; import org.infinispan.context.InvocationContext; +import org.infinispan.context.impl.LocalTxInvocationContext; +import org.infinispan.context.impl.RemoteTxInvocationContext; import org.infinispan.context.impl.TxInvocationContext; import org.infinispan.factories.annotations.Inject; import org.infinispan.interceptors.base.CommandInterceptor; @@ -44,13 +47,12 @@ import org.infinispan.jmx.annotations.ManagedAttribute; import org.infinispan.jmx.annotations.ManagedOperation; import org.infinispan.jmx.annotations.MeasurementType; +import org.infinispan.partitionhandling.impl.PartitionHandlingManager; import org.infinispan.remoting.rpc.RpcManager; import org.infinispan.remoting.transport.Address; import org.infinispan.statetransfer.OutdatedTopologyException; -import org.infinispan.transaction.LockingMode; import org.infinispan.transaction.impl.LocalTransaction; import org.infinispan.transaction.impl.RemoteTransaction; -import org.infinispan.transaction.impl.TransactionCoordinator; import org.infinispan.transaction.impl.TransactionTable; import org.infinispan.transaction.xa.GlobalTransaction; import org.infinispan.transaction.xa.recovery.RecoverableTransactionIdentifier; @@ -70,23 +72,24 @@ @MBean(objectName = "Transactions", description = "Component that manages the cache's participation in JTA transactions.") public class TxInterceptor extends CommandInterceptor implements JmxStatisticsExposer { - private TransactionTable txTable; + private static final Log log = LogFactory.getLog(TxInterceptor.class); + private static final boolean trace = log.isTraceEnabled(); private final AtomicLong prepares = new AtomicLong(0); private final AtomicLong commits = new AtomicLong(0); private final AtomicLong rollbacks = new AtomicLong(0); - private boolean statisticsEnabled; - protected TransactionCoordinator txCoordinator; - protected RpcManager rpcManager; - private static final Log log = LogFactory.getLog(TxInterceptor.class); - private static final boolean trace = log.isTraceEnabled(); + private RpcManager rpcManager; + private CommandsFactory commandsFactory; + private Cache cache; private RecoveryManager recoveryManager; + private TransactionTable txTable; + private PartitionHandlingManager partitionHandlingManager; + private boolean isTotalOrder; private boolean useOnePhaseForAutoCommitTx; private boolean useVersioning; - private CommandsFactory commandsFactory; - private Cache cache; + private boolean statisticsEnabled; @Override protected Log getLog() { @@ -94,21 +97,21 @@ protected Log getLog() { } @Inject - public void init(TransactionTable txTable, Configuration configuration, TransactionCoordinator txCoordinator, RpcManager rpcManager, - RecoveryManager recoveryManager, CommandsFactory commandsFactory, Cache cache) { + public void init(TransactionTable txTable, Configuration configuration, RpcManager rpcManager, + RecoveryManager recoveryManager, CommandsFactory commandsFactory, Cache cache, + PartitionHandlingManager partitionHandlingManager) { this.cacheConfiguration = configuration; this.txTable = txTable; - this.txCoordinator = txCoordinator; this.rpcManager = rpcManager; this.recoveryManager = recoveryManager; this.commandsFactory = commandsFactory; this.cache = cache; + this.partitionHandlingManager = partitionHandlingManager; statisticsEnabled = cacheConfiguration.jmxStatistics().enabled(); isTotalOrder = configuration.transaction().transactionProtocol().isTotalOrder(); useOnePhaseForAutoCommitTx = cacheConfiguration.transaction().use1PcForAutoCommitTransactions(); - useVersioning = configuration.transaction().transactionMode().isTransactional() && configuration.locking().writeSkewCheck() && - configuration.transaction().lockingMode() == LockingMode.OPTIMISTIC && configuration.versioning().enabled(); + useVersioning = Configurations.isVersioningEnabled(configuration); } @Override @@ -138,48 +141,7 @@ private Object invokeNextInterceptorAndVerifyTransaction(TxInvocationContext ctx return invokeNextInterceptor(ctx, command); } finally { if (!ctx.isOriginLocal()) { - // command.getOrigin() and ctx.getOrigin() are not reliable for LockControlCommands started by - // ClusteredGetCommands, or for PrepareCommands started by MultipleRpcCommands (when the replication queue - // is enabled). - Address origin = ctx.getGlobalTransaction().getAddress(); - //It is possible to receive a prepare or lock control command from a node that crashed. If that's the case rollback - //the transaction forcefully in order to cleanup resources. - boolean originatorMissing = !rpcManager.getTransport().getMembers().contains(origin); - // It is also possible that the LCC timed out on the originator's end and this node has processed - // a TxCompletionNotification. So we need to check the presence of the remote transaction to - // see if we need to clean up any acquired locks on our end. - boolean alreadyCompleted = txTable.isTransactionCompleted(command.getGlobalTransaction()) || - !txTable.containRemoteTx(command.getGlobalTransaction()); - // We want to throw an exception if the originator left the cluster and the transaction is not finished - // and/or it was rolled back by TransactionTable.cleanupLeaverTransactions(). - // We don't want to throw an exception if the originator left the cluster but the transaction already - // completed successfully. So far, this only seems possible when forcing the commit of an orphaned - // transaction (with recovery enabled). - boolean completedSuccessfully = alreadyCompleted && !ctx.getCacheTransaction().isMarkedForRollback(); - if (trace) { - log.tracef("invokeNextInterceptorAndVerifyTransaction :: originatorMissing=%s, alreadyCompleted=%s", - originatorMissing, alreadyCompleted); - } - - if (alreadyCompleted || originatorMissing) { - if (trace) { - log.tracef("Rolling back remote transaction %s because either already completed (%s) or originator no " + - "longer in the cluster (%s).", - command.getGlobalTransaction(), alreadyCompleted, originatorMissing); - } - RollbackCommand rollback = new RollbackCommand(command.getCacheName(), command.getGlobalTransaction()); - try { - invokeNextInterceptor(ctx, rollback); - } finally { - RemoteTransaction remoteTx = (RemoteTransaction) ctx.getCacheTransaction(); - remoteTx.markForRollback(true); - txTable.removeRemoteTransaction(command.getGlobalTransaction()); - } - } - - if (originatorMissing && !completedSuccessfully) { - throw log.orphanTransactionRolledBack(ctx.getGlobalTransaction()); - } + verifyRemoteTransaction((RemoteTxInvocationContext) ctx, command); } } } @@ -195,28 +157,7 @@ public Object visitCommitCommand(TxInvocationContext ctx, CommitCommand command) } if (!isTotalOrder) { - // If a commit is received for a transaction that doesn't have its 'lookedUpEntries' populated - // we know for sure this transaction is 2PC and was received via state transfer but the preceding PrepareCommand - // was not received by local node because it was executed on the previous key owners. We need to re-prepare - // the transaction on local node to ensure its locks are acquired and lookedUpEntries is properly populated. - RemoteTransaction remoteTx = (RemoteTransaction) ctx.getCacheTransaction(); - if (trace) { - log.tracef("Remote tx topology id %d and command topology is %d", remoteTx.lookedUpEntriesTopology(), - command.getTopologyId()); - } - if (remoteTx.lookedUpEntriesTopology() < command.getTopologyId()) { - PrepareCommand prepareCommand; - if (useVersioning) { - prepareCommand = commandsFactory.buildVersionedPrepareCommand(ctx.getGlobalTransaction(), ctx.getModifications(), false); - } else { - prepareCommand = commandsFactory.buildPrepareCommand(ctx.getGlobalTransaction(), ctx.getModifications(), false); - } - commandsFactory.initializeReplicableCommand(prepareCommand, true); - prepareCommand.setOrigin(ctx.getOrigin()); - if (trace) - log.tracef("Replaying the transactions received as a result of state transfer %s", prepareCommand); - visitPrepareCommand(ctx, prepareCommand); - } + replayRemoteTransactionIfNeeded((RemoteTxInvocationContext) ctx, command.getTopologyId()); } } @@ -240,9 +181,9 @@ public Object visitRollbackCommand(TxInvocationContext ctx, RollbackCommand comm } finally { //for tx that rollback we do not send a TxCompletionNotification, so we should cleanup // the recovery info here - if (recoveryManager!=null) { + if (recoveryManager != null) { GlobalTransaction gtx = command.getGlobalTransaction(); - recoveryManager.removeRecoveryInformation(((RecoverableTransactionIdentifier)gtx).getXid()); + recoveryManager.removeRecoveryInformation(((RecoverableTransactionIdentifier) gtx).getXid()); } } } @@ -332,10 +273,10 @@ public Object visitGetAllCommand(InvocationContext ctx, GetAllCommand command) t public EntryIterable visitEntryRetrievalCommand(InvocationContext ctx, EntryRetrievalCommand command) throws Throwable { // Enlistment shouldn't be needed for this command. The remove on the iterator will internally make a remove // command and the iterator itself does not place read values into the context. - EntryIterable iterable = (EntryIterable) super.visitEntryRetrievalCommand(ctx, command); + EntryIterable iterable = (EntryIterable) invokeNextInterceptor(ctx, command); if (ctx.isInTxScope()) { - return new TransactionAwareEntryIterable(iterable, command.getFilter(), - (TxInvocationContext) ctx, cache); + //noinspection unchecked + return new TransactionAwareEntryIterable(iterable, command.getFilter(), (LocalTxInvocationContext) ctx, cache); } else { return iterable; } @@ -461,4 +402,81 @@ public long getCommits() { public long getRollbacks() { return rollbacks.get(); } + + private void verifyRemoteTransaction(RemoteTxInvocationContext ctx, AbstractTransactionBoundaryCommand command) throws Throwable { + final GlobalTransaction globalTransaction = command.getGlobalTransaction(); + + // command.getOrigin() and ctx.getOrigin() are not reliable for LockControlCommands started by + // ClusteredGetCommands, or for PrepareCommands started by MultipleRpcCommands (when the replication queue + // is enabled). + final Address origin = globalTransaction.getAddress(); + + //It is possible to receive a prepare or lock control command from a node that crashed. If that's the case rollback + //the transaction forcefully in order to cleanup resources. + boolean originatorMissing = !rpcManager.getTransport().getMembers().contains(origin); + + // It is also possible that the LCC timed out on the originator's end and this node has processed + // a TxCompletionNotification. So we need to check the presence of the remote transaction to + // see if we need to clean up any acquired locks on our end. + boolean alreadyCompleted = txTable.isTransactionCompleted(globalTransaction) || !txTable.containRemoteTx(globalTransaction); + + // We want to throw an exception if the originator left the cluster and the transaction is not finished + // and/or it was rolled back by TransactionTable.cleanupLeaverTransactions(). + // We don't want to throw an exception if the originator left the cluster but the transaction already + // completed successfully. So far, this only seems possible when forcing the commit of an orphaned + // transaction (with recovery enabled). + boolean completedSuccessfully = alreadyCompleted && !ctx.getCacheTransaction().isMarkedForRollback(); + + boolean canRollback = command instanceof PrepareCommand && !((PrepareCommand) command).isOnePhaseCommit() || + command instanceof RollbackCommand || command instanceof LockControlCommand; + + if (trace) { + log.tracef("invokeNextInterceptorAndVerifyTransaction :: originatorMissing=%s, alreadyCompleted=%s", + originatorMissing, alreadyCompleted); + } + + if (alreadyCompleted || (originatorMissing && (canRollback || partitionHandlingManager.canRollbackTransactionAfterOriginatorLeave(globalTransaction)))) { + if (trace) { + log.tracef("Rolling back remote transaction %s because either already completed (%s) or originator no longer in the cluster (%s).", + globalTransaction, alreadyCompleted, originatorMissing); + } + RollbackCommand rollback = commandsFactory.buildRollbackCommand(command.getGlobalTransaction()); + try { + invokeNextInterceptor(ctx, rollback); + } finally { + RemoteTransaction remoteTx = ctx.getCacheTransaction(); + remoteTx.markForRollback(true); + txTable.removeRemoteTransaction(globalTransaction); + } + + if (originatorMissing && !completedSuccessfully) { + throw log.orphanTransactionRolledBack(globalTransaction); + } + } + } + + private void replayRemoteTransactionIfNeeded(RemoteTxInvocationContext ctx, int topologyId) throws Throwable { + // If a commit is received for a transaction that doesn't have its 'lookedUpEntries' populated + // we know for sure this transaction is 2PC and was received via state transfer but the preceding PrepareCommand + // was not received by local node because it was executed on the previous key owners. We need to re-prepare + // the transaction on local node to ensure its locks are acquired and lookedUpEntries is properly populated. + RemoteTransaction remoteTx = ctx.getCacheTransaction(); + if (trace) { + log.tracef("Remote tx topology id %d and command topology is %d", remoteTx.lookedUpEntriesTopology(), topologyId); + } + if (remoteTx.lookedUpEntriesTopology() < topologyId) { + PrepareCommand prepareCommand; + if (useVersioning) { + prepareCommand = commandsFactory.buildVersionedPrepareCommand(ctx.getGlobalTransaction(), ctx.getModifications(), false); + } else { + prepareCommand = commandsFactory.buildPrepareCommand(ctx.getGlobalTransaction(), ctx.getModifications(), false); + } + commandsFactory.initializeReplicableCommand(prepareCommand, true); + prepareCommand.setOrigin(ctx.getOrigin()); + if (trace) { + log.tracef("Replaying the transactions received as a result of state transfer %s", prepareCommand); + } + visitPrepareCommand(ctx, prepareCommand); + } + } } 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 312926121ae..3b62245377e 100644 --- a/core/src/main/java/org/infinispan/interceptors/distribution/BaseDistributionInterceptor.java +++ b/core/src/main/java/org/infinispan/interceptors/distribution/BaseDistributionInterceptor.java @@ -1,7 +1,6 @@ package org.infinispan.interceptors.distribution; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -267,11 +266,9 @@ protected final Object handleNonTxWriteCommand(InvocationContext ctx, DataWriteC throw new CacheException("Attempted execution of non-transactional write command in a transactional invocation context"); } - RecipientGenerator recipientGenerator = new SingleKeyRecipientGenerator(command.getKey()); - // see if we need to load values from remote sources first if (needValuesFromPreviousOwners(ctx, command)) { - remoteGetBeforeWrite(ctx, command, recipientGenerator); + remoteGetBeforeWrite(ctx, command, command.getKey()); } // invoke the command locally, we need to know if it's successful or not @@ -309,7 +306,7 @@ protected final Object handleNonTxWriteCommand(InvocationContext ctx, DataWriteC log.tracef("Skipping the replication of the conditional command as it did not succeed on primary owner (%s).", command); return localResult; } - List
recipients = recipientGenerator.generateRecipients(); + List
recipients = cdl.getOwners(command.getKey()); // Ignore the previous value on the backup owners command.setValueMatcher(ValueMatcher.MATCH_ALWAYS); try { @@ -327,7 +324,7 @@ protected final Object handleNonTxWriteCommand(InvocationContext ctx, DataWriteC log.tracef("Skipping the replication of the command as it did not succeed on primary owner (%s).", command); return localResult; } - List
recipients = recipientGenerator.generateRecipients(); + List
recipients = cdl.getOwners(command.getKey()); log.tracef("I'm the primary owner, sending the command to all the backups (%s) in order to be applied.", recipients); // check if a single owner has been configured and the target for the key is the local address @@ -517,59 +514,5 @@ public Object visitGetAllCommand(InvocationContext ctx, GetAllCommand command) t */ protected abstract boolean needValuesFromPreviousOwners(InvocationContext ctx, WriteCommand command); - protected abstract void remoteGetBeforeWrite(InvocationContext ctx, WriteCommand command, RecipientGenerator keygen) throws Throwable; - - interface RecipientGenerator { - - Collection getKeys(); - - List
generateRecipients(); - } - - class SingleKeyRecipientGenerator implements RecipientGenerator { - private final Object key; - private final Set keys; - private List
recipients = null; - - SingleKeyRecipientGenerator(Object key) { - this.key = key; - keys = Collections.singleton(key); - } - - @Override - public List
generateRecipients() { - if (recipients == null) { - recipients = cdl.getOwners(key); - } - return recipients; - } - - @Override - public Collection getKeys() { - return keys; - } - } - - class MultipleKeysRecipientGenerator implements RecipientGenerator { - - private final Collection keys; - private List
recipients = null; - - MultipleKeysRecipientGenerator(Collection keys) { - this.keys = keys; - } - - @Override - public List
generateRecipients() { - if (recipients == null) { - recipients = cdl.getOwners(keys); - } - return recipients; - } - - @Override - public Collection getKeys() { - return keys; - } - } + protected abstract void remoteGetBeforeWrite(InvocationContext ctx, WriteCommand command, Object key) throws Throwable; } diff --git a/core/src/main/java/org/infinispan/interceptors/distribution/NonTxDistributionInterceptor.java b/core/src/main/java/org/infinispan/interceptors/distribution/NonTxDistributionInterceptor.java index 8e226f3b455..ff02d53b7e6 100644 --- a/core/src/main/java/org/infinispan/interceptors/distribution/NonTxDistributionInterceptor.java +++ b/core/src/main/java/org/infinispan/interceptors/distribution/NonTxDistributionInterceptor.java @@ -204,13 +204,11 @@ public Object visitReplaceCommand(InvocationContext ctx, ReplaceCommand command) return handleNonTxWriteCommand(ctx, command); } - protected void remoteGetBeforeWrite(InvocationContext ctx, WriteCommand command, RecipientGenerator keygen) throws Throwable { - for (Object k : keygen.getKeys()) { - if (cdl.localNodeIsPrimaryOwner(k)) { - // Then it makes sense to try a local get and wrap again. This will compensate the fact the the entry was not local - // earlier when the EntryWrappingInterceptor executed during current invocation context but it should be now. - localGetCacheEntry(ctx, k, true, command); - } + protected void remoteGetBeforeWrite(InvocationContext ctx, WriteCommand command, Object key) throws Throwable { + if (cdl.localNodeIsPrimaryOwner(key)) { + // Then it makes sense to try a local get and wrap again. This will compensate the fact the the entry was not local + // earlier when the EntryWrappingInterceptor executed during current invocation context but it should be now. + localGetCacheEntry(ctx, key, true, command); } } diff --git a/core/src/main/java/org/infinispan/interceptors/distribution/TxDistributionInterceptor.java b/core/src/main/java/org/infinispan/interceptors/distribution/TxDistributionInterceptor.java index 30e36b340e4..5f7186d6f1e 100644 --- a/core/src/main/java/org/infinispan/interceptors/distribution/TxDistributionInterceptor.java +++ b/core/src/main/java/org/infinispan/interceptors/distribution/TxDistributionInterceptor.java @@ -1,30 +1,32 @@ package org.infinispan.interceptors.distribution; import org.infinispan.commands.FlagAffectedCommand; -import org.infinispan.commands.TopologyAffectedCommand; import org.infinispan.commands.control.LockControlCommand; import org.infinispan.commands.read.AbstractDataCommand; import org.infinispan.commands.read.GetCacheEntryCommand; import org.infinispan.commands.read.GetKeyValueCommand; -import org.infinispan.commands.read.GetAllCommand; import org.infinispan.commands.tx.CommitCommand; import org.infinispan.commands.tx.PrepareCommand; import org.infinispan.commands.tx.RollbackCommand; import org.infinispan.commands.tx.TransactionBoundaryCommand; +import org.infinispan.commands.tx.VersionedCommitCommand; import org.infinispan.commands.write.PutKeyValueCommand; import org.infinispan.commands.write.PutMapCommand; import org.infinispan.commands.write.RemoveCommand; import org.infinispan.commands.write.ReplaceCommand; import org.infinispan.commands.write.ValueMatcher; import org.infinispan.commands.write.WriteCommand; +import org.infinispan.configuration.cache.Configurations; import org.infinispan.container.entries.CacheEntry; import org.infinispan.container.entries.InternalCacheEntry; +import org.infinispan.container.versioning.EntryVersionsMap; import org.infinispan.context.Flag; import org.infinispan.context.InvocationContext; import org.infinispan.context.impl.LocalTxInvocationContext; import org.infinispan.context.impl.TxInvocationContext; -import org.infinispan.distribution.ch.ConsistentHash; +import org.infinispan.factories.annotations.Inject; import org.infinispan.factories.annotations.Start; +import org.infinispan.partitionhandling.impl.PartitionHandlingManager; import org.infinispan.remoting.inboundhandler.DeliverOrder; import org.infinispan.remoting.responses.CacheNotFoundResponse; import org.infinispan.remoting.responses.Response; @@ -36,20 +38,16 @@ import org.infinispan.statetransfer.OutdatedTopologyException; import org.infinispan.transaction.LockingMode; import org.infinispan.transaction.impl.LocalTransaction; +import org.infinispan.transaction.xa.GlobalTransaction; import org.infinispan.util.logging.Log; import org.infinispan.util.logging.LogFactory; -import org.jgroups.demos.Topology; +import java.util.Arrays; import java.util.Collection; -import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; -import java.util.concurrent.TimeoutException; -import static org.infinispan.commons.util.Util.toStr; +import static java.lang.String.format; import static org.infinispan.util.DeltaCompositeKeyUtil.filterDeltaCompositeKey; import static org.infinispan.util.DeltaCompositeKeyUtil.filterDeltaCompositeKeys; import static org.infinispan.util.DeltaCompositeKeyUtil.getAffectedKeysFromContext; @@ -65,13 +63,26 @@ public class TxDistributionInterceptor extends BaseDistributionInterceptor { private static Log log = LogFactory.getLog(TxDistributionInterceptor.class); private static final boolean trace = log.isTraceEnabled(); + private PartitionHandlingManager partitionHandlingManager; + private boolean isPessimisticCache; private boolean useClusteredWriteSkewCheck; + @Inject + public void inject(PartitionHandlingManager partitionHandlingManager) { + this.partitionHandlingManager = partitionHandlingManager; + } + + @Start + public void start() { + isPessimisticCache = cacheConfiguration.transaction().lockingMode() == LockingMode.PESSIMISTIC; + useClusteredWriteSkewCheck = Configurations.isVersioningEnabled(cacheConfiguration); + } + @Override public Object visitReplaceCommand(InvocationContext ctx, ReplaceCommand command) throws Throwable { try { - return handleTxWriteCommand(ctx, command, new SingleKeyRecipientGenerator(command.getKey()), false); + return handleTxWriteCommand(ctx, command, command.getKey(), false); } finally { if (ctx.isOriginLocal()) { // If the state transfer interceptor has to retry the command, it should ignore the previous value. @@ -83,7 +94,7 @@ public Object visitReplaceCommand(InvocationContext ctx, ReplaceCommand command) @Override public Object visitRemoveCommand(InvocationContext ctx, RemoveCommand command) throws Throwable { try { - return handleTxWriteCommand(ctx, command, new SingleKeyRecipientGenerator(command.getKey()), false); + return handleTxWriteCommand(ctx, command, command.getKey(), false); } finally { if (ctx.isOriginLocal()) { // If the state transfer interceptor has to retry the command, it should ignore the previous value. @@ -92,21 +103,13 @@ public Object visitRemoveCommand(InvocationContext ctx, RemoveCommand command) t } } - @Start - public void start() { - isPessimisticCache = cacheConfiguration.transaction().lockingMode() == LockingMode.PESSIMISTIC; - useClusteredWriteSkewCheck = !isPessimisticCache && - cacheConfiguration.versioning().enabled() && cacheConfiguration.locking().writeSkewCheck(); - } - @Override public Object visitPutKeyValueCommand(InvocationContext ctx, PutKeyValueCommand command) throws Throwable { if (command.hasFlag(Flag.PUT_FOR_EXTERNAL_READ)) { return handleNonTxWriteCommand(ctx, command); } - SingleKeyRecipientGenerator skrg = new SingleKeyRecipientGenerator(command.getKey()); - Object returnValue = handleTxWriteCommand(ctx, command, skrg, command.hasFlag(Flag.PUT_FOR_STATE_TRANSFER)); + Object returnValue = handleTxWriteCommand(ctx, command, command.getKey(), command.hasFlag(Flag.PUT_FOR_STATE_TRANSFER)); if (ctx.isOriginLocal()) { // If the state transfer interceptor has to retry the command, it should ignore the previous value. command.setValueMatcher(command.isSuccessful() ? ValueMatcher.MATCH_ALWAYS : ValueMatcher.MATCH_NEVER); @@ -117,7 +120,7 @@ public Object visitPutKeyValueCommand(InvocationContext ctx, PutKeyValueCommand @Override public Object visitPutMapCommand(InvocationContext ctx, PutMapCommand command) throws Throwable { // don't bother with a remote get for the PutMapCommand! - return handleTxWriteCommand(ctx, command, new MultipleKeysRecipientGenerator(command.getMap().keySet()), true); + return handleTxWriteCommand(ctx, command, null, true); } @Override @@ -166,13 +169,6 @@ private Object visitGetCommand(InvocationContext ctx, AbstractDataCommand comman return returnValue; } - protected void lockAndWrap(InvocationContext ctx, Object key, InternalCacheEntry ice, FlagAffectedCommand command) throws InterruptedException { - boolean skipLocking = hasSkipLocking(command); - long lockTimeout = getLockAcquisitionTimeout(command, skipLocking); - lockManager.acquireLock(ctx, key, lockTimeout, skipLocking); - entryFactory.wrapEntryForPut(ctx, key, ice, false, command, false); - } - @Override public Object visitLockControlCommand(TxInvocationContext ctx, LockControlCommand command) throws Throwable { if (ctx.isOriginLocal()) { @@ -183,7 +179,7 @@ public Object visitLockControlCommand(TxInvocationContext ctx, LockControlComman log.tracef("Registered remote locks acquired %s", affectedNodes); RpcOptions rpcOptions = rpcManager.getRpcOptionsBuilder(ResponseMode.SYNCHRONOUS_IGNORE_LEAVERS, DeliverOrder.NONE).build(); Map responseMap = rpcManager.invokeRemotely(affectedNodes, command, rpcOptions); - checkTxCommandResponses(responseMap, command); + checkTxCommandResponses(responseMap, command, (LocalTxInvocationContext) ctx, ((LocalTxInvocationContext) ctx).getRemoteLocksAcquired()); } return invokeNextInterceptor(ctx, command); } @@ -192,7 +188,9 @@ public Object visitLockControlCommand(TxInvocationContext ctx, LockControlComman @Override public Object visitCommitCommand(TxInvocationContext ctx, CommitCommand command) throws Throwable { if (shouldInvokeRemoteTxCommand(ctx)) { - sendCommitCommand(ctx, command); + Collection
recipients = getCommitNodes(ctx); + Map responseMap = rpcManager.invokeRemotely(recipients, command, createCommitRpcOptions()); + checkTxCommandResponses(responseMap, command, (LocalTxInvocationContext) ctx, recipients); } return invokeNextInterceptor(ctx, command); } @@ -203,24 +201,18 @@ public Object visitPrepareCommand(TxInvocationContext ctx, PrepareCommand comman if (shouldInvokeRemoteTxCommand(ctx)) { Collection
recipients = cdl.getOwners(getAffectedKeysFromContext(ctx)); - prepareOnAffectedNodes(ctx, command, recipients, defaultSynchronous); + prepareOnAffectedNodes(ctx, command, recipients); ((LocalTxInvocationContext) ctx).remoteLocksAcquired(recipients == null ? dm.getWriteConsistentHash() .getMembers() : recipients); } return retVal; } - protected void prepareOnAffectedNodes(TxInvocationContext ctx, PrepareCommand command, Collection
recipients, boolean sync) { + protected void prepareOnAffectedNodes(TxInvocationContext ctx, PrepareCommand command, Collection
recipients) { try { // this method will return immediately if we're the only member (because exclude_self=true) - RpcOptions rpcOptions; - if (sync) { - rpcOptions = rpcManager.getRpcOptionsBuilder(ResponseMode.SYNCHRONOUS_IGNORE_LEAVERS, DeliverOrder.NONE).build(); - } else { - rpcOptions = rpcManager.getDefaultRpcOptions(false); - } - Map responseMap = rpcManager.invokeRemotely(recipients, command, rpcOptions); - checkTxCommandResponses(responseMap, command); + Map responseMap = rpcManager.invokeRemotely(recipients, command, createPrepareRpcOptions()); + checkTxCommandResponses(responseMap, command, (LocalTxInvocationContext) ctx, recipients); } finally { transactionRemotelyPrepared(ctx); } @@ -229,12 +221,9 @@ protected void prepareOnAffectedNodes(TxInvocationContext ctx, PrepareCommand @Override public Object visitRollbackCommand(TxInvocationContext ctx, RollbackCommand command) throws Throwable { if (shouldInvokeRemoteTxCommand(ctx)) { - boolean syncRollback = cacheConfiguration.transaction().syncRollbackPhase(); - ResponseMode responseMode = syncRollback ? ResponseMode.SYNCHRONOUS_IGNORE_LEAVERS : ResponseMode.ASYNCHRONOUS; - RpcOptions rpcOptions = rpcManager.getRpcOptionsBuilder(responseMode, DeliverOrder.NONE).build(); Collection
recipients = getCommitNodes(ctx); - Map responseMap = rpcManager.invokeRemotely(recipients, command, rpcOptions); - checkTxCommandResponses(responseMap, command); + Map responseMap = rpcManager.invokeRemotely(recipients, command, createRollbackRpcOptions()); + checkTxCommandResponses(responseMap, command, (LocalTxInvocationContext) ctx, recipients); } return invokeNextInterceptor(ctx, command); @@ -247,47 +236,68 @@ private Collection
getCommitNodes(TxInvocationContext ctx) { return localTx.getCommitNodes(affectedNodes, rpcManager.getTopologyId(), members); } - private void sendCommitCommand(TxInvocationContext ctx, CommitCommand command) throws TimeoutException, InterruptedException { - Collection
recipients = getCommitNodes(ctx); - boolean syncCommitPhase = cacheConfiguration.transaction().syncCommitPhase(); - RpcOptions rpcOptions; - if (syncCommitPhase) { - rpcOptions = rpcManager.getRpcOptionsBuilder(ResponseMode.SYNCHRONOUS_IGNORE_LEAVERS, DeliverOrder.NONE).build(); - } else { - rpcOptions = rpcManager.getDefaultRpcOptions(false, DeliverOrder.NONE); - } - Map responseMap = rpcManager.invokeRemotely(recipients, command, rpcOptions); - checkTxCommandResponses(responseMap, command); - } - - protected void checkTxCommandResponses(Map responseMap, TransactionBoundaryCommand command) { + protected void checkTxCommandResponses(Map responseMap, TransactionBoundaryCommand command, + LocalTxInvocationContext context, Collection
recipients) { + OutdatedTopologyException outdatedTopologyException = null; for (Map.Entry e : responseMap.entrySet()) { Address recipient = e.getKey(); Response response = e.getValue(); - if (response instanceof CacheNotFoundResponse) { + if (response == CacheNotFoundResponse.INSTANCE) { // No need to retry if the missing node wasn't a member when the command started. if (command.getTopologyId() == stateTransferManager.getCacheTopology().getTopologyId() && !rpcManager.getMembers().contains(recipient)) { - log.tracef("Ignoring response from node not targeted %s", recipient); + if (trace) log.tracef("Ignoring response from node not targeted %s", recipient); } else { - log.tracef("Cache not running on node %s, or the node is missing", recipient); - throw new OutdatedTopologyException("Cache not running on node " + recipient); + if (checkCacheNotFoundResponseInPartitionHandling(command, context, recipients)) { + if (trace) log.tracef("Cache not running on node %s, or the node is missing. It will be handled by the PartitionHandlingManager", recipient); + return; + } else { + if (trace) log.tracef("Cache not running on node %s, or the node is missing", recipient); + //noinspection ThrowableInstanceNeverThrown + outdatedTopologyException = new OutdatedTopologyException(format("Cache not running on node %s, or the node is missing", recipient)); + } } - } else if (response instanceof UnsureResponse) { - log.tracef("Node %s has a newer topology id", recipient); - throw new OutdatedTopologyException("Cache not running on node " + recipient); + } else if (response == UnsureResponse.INSTANCE) { + if (trace) log.tracef("Node %s has a newer topology id", recipient); + //noinspection ThrowableInstanceNeverThrown + outdatedTopologyException = new OutdatedTopologyException(format("Node %s has a newer topology id", recipient)); } } + if (outdatedTopologyException != null) { + throw outdatedTopologyException; + } + } + + private boolean checkCacheNotFoundResponseInPartitionHandling(TransactionBoundaryCommand command, + LocalTxInvocationContext context, + Collection
recipients) { + final GlobalTransaction globalTransaction = command.getGlobalTransaction(); + final Collection lockedKeys = context.getLockedKeys(); + if (command instanceof RollbackCommand) { + return partitionHandlingManager.addPartialRollbackTransaction(globalTransaction, recipients, lockedKeys); + } else if (command instanceof PrepareCommand) { + if (((PrepareCommand) command).isOnePhaseCommit()) { + return partitionHandlingManager.addPartialCommit1PCTransaction(globalTransaction, recipients, lockedKeys, + Arrays.asList(((PrepareCommand) command).getModifications())); + } + } else if (command instanceof CommitCommand) { + EntryVersionsMap newVersion = null; + if (command instanceof VersionedCommitCommand) { + newVersion = ((VersionedCommitCommand) command).getUpdatedVersions(); + } + return partitionHandlingManager.addPartialCommit2PCTransaction(globalTransaction, recipients, lockedKeys, newVersion); + } + return false; } private boolean shouldFetchRemoteValuesForWriteSkewCheck(InvocationContext ctx, WriteCommand cmd) { // Note: the primary owner always already has the data, so this method is always going to return false if (useClusteredWriteSkewCheck && ctx.isInTxScope() && dm.isRehashInProgress()) { for (Object key : cmd.getAffectedKeys()) { - // TODO Dan: Do we need a special check for total order? boolean shouldPerformWriteSkewCheck = cdl.localNodeIsPrimaryOwner(key); // TODO Dan: remoteGet() already checks if the key is available locally or not - if (shouldPerformWriteSkewCheck && dm.isAffectedByRehash(key) && !dataContainer.containsKey(key)) return true; + if (shouldPerformWriteSkewCheck && dm.isAffectedByRehash(key) && !dataContainer.containsKey(key)) + return true; } } return false; @@ -297,10 +307,10 @@ private boolean shouldFetchRemoteValuesForWriteSkewCheck(InvocationContext ctx, * If we are within one transaction we won't do any replication as replication would only be performed at commit * time. If the operation didn't originate locally we won't do any replication either. */ - private Object handleTxWriteCommand(InvocationContext ctx, WriteCommand command, RecipientGenerator recipientGenerator, boolean skipRemoteGet) throws Throwable { + private Object handleTxWriteCommand(InvocationContext ctx, WriteCommand command, Object key, boolean skipRemoteGet) throws Throwable { // see if we need to load values from remote sources first if (!skipRemoteGet && needValuesFromPreviousOwners(ctx, command)) - remoteGetBeforeWrite(ctx, command, recipientGenerator); + remoteGetBeforeWrite(ctx, command, key); // FIRST pass this call up the chain. Only if it succeeds (no exceptions) locally do we attempt to distribute. return invokeNextInterceptor(ctx, command); @@ -318,7 +328,7 @@ protected boolean needValuesFromPreviousOwners(InvocationContext ctx, WriteComma } private Object localGet(InvocationContext ctx, Object key, boolean isWrite, - FlagAffectedCommand command, boolean isGetCacheEntry) throws Throwable { + FlagAffectedCommand command, boolean isGetCacheEntry) throws Throwable { InternalCacheEntry ice = fetchValueLocallyIfAvailable(dm.getReadConsistentHash(), key); if (ice != null) { if (isWrite && isPessimisticCache && ctx.isInTxScope()) { @@ -326,7 +336,7 @@ private Object localGet(InvocationContext ctx, Object key, boolean isWrite, } if (!ctx.replaceValue(key, ice)) { if (isWrite) - lockAndWrap(ctx, key, ice, command); + entryFactory.wrapEntryForPut(ctx, key, ice, false, command, false); else ctx.putLookedUpEntry(key, ice); } @@ -335,17 +345,15 @@ private Object localGet(InvocationContext ctx, Object key, boolean isWrite, return null; } - protected void remoteGetBeforeWrite(InvocationContext ctx, WriteCommand command, RecipientGenerator keygen) throws Throwable { - for (Object k : keygen.getKeys()) { - CacheEntry entry = ctx.lookupEntry(k); - boolean skipRemoteGet = entry != null && entry.skipLookup(); - if (skipRemoteGet) { - continue; - } - InternalCacheEntry ice = remoteGet(ctx, k, true, command); - if (ice == null) { - localGet(ctx, k, true, command, false); - } + protected void remoteGetBeforeWrite(InvocationContext ctx, WriteCommand command, Object key) throws Throwable { + CacheEntry entry = ctx.lookupEntry(key); + boolean skipRemoteGet = entry != null && entry.skipLookup(); + if (skipRemoteGet) { + return; + } + InternalCacheEntry ice = remoteGet(ctx, key, true, command); + if (ice == null) { + localGet(ctx, key, true, command, false); } } @@ -367,12 +375,12 @@ private InternalCacheEntry remoteGet(InvocationContext ctx, Object key, boolean if (ice != null) { if (useClusteredWriteSkewCheck && ctx.isInTxScope()) { - ((TxInvocationContext)ctx).getCacheTransaction().putLookedUpRemoteVersion(key, ice.getMetadata().version()); + ((TxInvocationContext) ctx).getCacheTransaction().putLookedUpRemoteVersion(key, ice.getMetadata().version()); } if (!ctx.replaceValue(key, ice)) { if (isWrite) - lockAndWrap(ctx, key, ice, command); + entryFactory.wrapEntryForPut(ctx, key, ice, false, command, false); else { ctx.putLookedUpEntry(key, ice); if (ctx.isInTxScope()) { @@ -383,8 +391,32 @@ private InternalCacheEntry remoteGet(InvocationContext ctx, Object key, boolean return ice; } } else { - if (trace) log.tracef("Not doing a remote get for key %s since entry is mapped to current node (%s), or is in L1. Owners are %s", key, rpcManager.getAddress(), dm.locate(key)); + if (trace) { + log.tracef("Not doing a remote get for key %s since entry is mapped to current node (%s), or is in L1. Owners are %s", key, rpcManager.getAddress(), dm.locate(key)); + } } return null; } + + private RpcOptions createCommitRpcOptions() { + return createRpcOptionsFor2ndPhase(cacheConfiguration.transaction().syncCommitPhase()); + } + + private RpcOptions createRollbackRpcOptions() { + return createRpcOptionsFor2ndPhase(cacheConfiguration.transaction().syncRollbackPhase()); + } + + private RpcOptions createRpcOptionsFor2ndPhase(boolean sync) { + if (sync) { + return rpcManager.getRpcOptionsBuilder(ResponseMode.SYNCHRONOUS_IGNORE_LEAVERS, DeliverOrder.NONE).build(); + } else { + return rpcManager.getRpcOptionsBuilder(ResponseMode.ASYNCHRONOUS, DeliverOrder.NONE).build(); + } + } + + protected RpcOptions createPrepareRpcOptions() { + return cacheConfiguration.clustering().cacheMode().isSynchronous() ? + rpcManager.getRpcOptionsBuilder(ResponseMode.SYNCHRONOUS_IGNORE_LEAVERS, DeliverOrder.NONE).build() : + rpcManager.getDefaultRpcOptions(false); + } } diff --git a/core/src/main/java/org/infinispan/interceptors/distribution/VersionedDistributionInterceptor.java b/core/src/main/java/org/infinispan/interceptors/distribution/VersionedDistributionInterceptor.java index 45a46563c26..46daabc8a4e 100644 --- a/core/src/main/java/org/infinispan/interceptors/distribution/VersionedDistributionInterceptor.java +++ b/core/src/main/java/org/infinispan/interceptors/distribution/VersionedDistributionInterceptor.java @@ -1,11 +1,9 @@ package org.infinispan.interceptors.distribution; import org.infinispan.commands.tx.PrepareCommand; +import org.infinispan.context.impl.LocalTxInvocationContext; import org.infinispan.context.impl.TxInvocationContext; -import org.infinispan.remoting.inboundhandler.DeliverOrder; import org.infinispan.remoting.responses.Response; -import org.infinispan.remoting.rpc.ResponseMode; -import org.infinispan.remoting.rpc.RpcOptions; import org.infinispan.remoting.transport.Address; import org.infinispan.transaction.xa.CacheTransaction; import org.infinispan.util.logging.Log; @@ -32,12 +30,11 @@ protected Log getLog() { } @Override - protected void prepareOnAffectedNodes(TxInvocationContext ctx, PrepareCommand command, Collection
recipients, boolean ignored) { + protected void prepareOnAffectedNodes(TxInvocationContext ctx, PrepareCommand command, Collection
recipients) { // Perform the RPC try { - RpcOptions rpcOptions = rpcManager.getRpcOptionsBuilder(ResponseMode.SYNCHRONOUS_IGNORE_LEAVERS, DeliverOrder.NONE).build(); - Map resps = rpcManager.invokeRemotely(recipients, command, rpcOptions); - checkTxCommandResponses(resps, command); + Map resps = rpcManager.invokeRemotely(recipients, command, createPrepareRpcOptions()); + checkTxCommandResponses(resps, command, (LocalTxInvocationContext) ctx, recipients); // Now store newly generated versions from lock owners for use during the commit phase. CacheTransaction ct = ctx.getCacheTransaction(); diff --git a/core/src/main/java/org/infinispan/interceptors/locking/AbstractTxLockingInterceptor.java b/core/src/main/java/org/infinispan/interceptors/locking/AbstractTxLockingInterceptor.java index da5bd07dc90..21cea5d236c 100644 --- a/core/src/main/java/org/infinispan/interceptors/locking/AbstractTxLockingInterceptor.java +++ b/core/src/main/java/org/infinispan/interceptors/locking/AbstractTxLockingInterceptor.java @@ -1,14 +1,8 @@ package org.infinispan.interceptors.locking; -import static org.infinispan.commons.util.Util.toStr; - -import java.util.Collection; -import java.util.concurrent.TimeUnit; - import org.infinispan.atomic.DeltaCompositeKey; -import org.infinispan.commands.read.GetCacheEntryCommand; +import org.infinispan.commands.CommandsFactory; import org.infinispan.commands.read.GetAllCommand; -import org.infinispan.commands.read.GetKeyValueCommand; import org.infinispan.commands.tx.CommitCommand; import org.infinispan.commands.tx.PrepareCommand; import org.infinispan.commands.tx.RollbackCommand; @@ -18,6 +12,7 @@ import org.infinispan.context.InvocationContext; import org.infinispan.context.impl.TxInvocationContext; import org.infinispan.factories.annotations.Inject; +import org.infinispan.partitionhandling.impl.PartitionHandlingManager; import org.infinispan.remoting.rpc.RpcManager; import org.infinispan.statetransfer.OutdatedTopologyException; import org.infinispan.transaction.impl.LocalTransaction; @@ -28,6 +23,11 @@ import org.infinispan.util.concurrent.TimeoutException; import org.infinispan.util.logging.Log; +import java.util.Collection; +import java.util.concurrent.TimeUnit; + +import static org.infinispan.commons.util.Util.toStr; + /** * Base class for transaction based locking interceptors. * @@ -40,14 +40,16 @@ public abstract class AbstractTxLockingInterceptor extends AbstractLockingInterc protected RpcManager rpcManager; private boolean clustered; private TimeService timeService; + private PartitionHandlingManager partitionHandlingManager; @Inject - @SuppressWarnings("unused") - public void setDependencies(TransactionTable txTable, RpcManager rpcManager, TimeService timeService) { + public void setDependencies(TransactionTable txTable, RpcManager rpcManager, TimeService timeService, + PartitionHandlingManager partitionHandlingManager, CommandsFactory commandsFactory) { this.txTable = txTable; this.rpcManager = rpcManager; clustered = rpcManager != null; this.timeService = timeService; + this.partitionHandlingManager = partitionHandlingManager; } @Override @@ -210,17 +212,13 @@ private void waitForTransactionsToComplete(TxInvocationContext txContext, Collec } } - private TimeoutException newTimeoutException(Object key, TxInvocationContext txContext) { - return new TimeoutException("Could not acquire lock on " + key + " on behalf of transaction " + - txContext.getGlobalTransaction() + "." + "Lock is being held by " + lockManager.getOwner(key)); - } - - private TimeoutException newTimeoutException(Object key, CacheTransaction tx, TxInvocationContext txContext) { + private TimeoutException newTimeoutException(Object key, CacheTransaction tx, TxInvocationContext txContext) { return new TimeoutException("Could not acquire lock on " + key + " on behalf of transaction " + txContext.getGlobalTransaction() + ". Waiting to complete tx: " + tx + "."); } private boolean releaseLockOnTxCompletion(TxInvocationContext ctx) { - return ctx.isOriginLocal() || Configurations.isSecondPhaseAsync(cacheConfiguration); + return (ctx.isOriginLocal() && !partitionHandlingManager.isTransactionPartiallyCommitted(ctx.getGlobalTransaction()) || + (!ctx.isOriginLocal() && Configurations.isSecondPhaseAsync(cacheConfiguration))); } } diff --git a/core/src/main/java/org/infinispan/interceptors/totalorder/TotalOrderDistributionInterceptor.java b/core/src/main/java/org/infinispan/interceptors/totalorder/TotalOrderDistributionInterceptor.java index 465ba69b038..7da25cfd911 100644 --- a/core/src/main/java/org/infinispan/interceptors/totalorder/TotalOrderDistributionInterceptor.java +++ b/core/src/main/java/org/infinispan/interceptors/totalorder/TotalOrderDistributionInterceptor.java @@ -1,12 +1,9 @@ package org.infinispan.interceptors.totalorder; -import org.infinispan.commands.FlagAffectedCommand; import org.infinispan.commands.tx.CommitCommand; import org.infinispan.commands.tx.PrepareCommand; import org.infinispan.commands.tx.RollbackCommand; import org.infinispan.configuration.cache.Configurations; -import org.infinispan.container.entries.InternalCacheEntry; -import org.infinispan.context.InvocationContext; import org.infinispan.context.impl.TxInvocationContext; import org.infinispan.interceptors.distribution.TxDistributionInterceptor; import org.infinispan.remoting.transport.Address; @@ -46,8 +43,7 @@ public Object visitCommitCommand(TxInvocationContext ctx, CommitCommand command) } @Override - protected void prepareOnAffectedNodes(TxInvocationContext ctx, PrepareCommand command, Collection
recipients, - boolean sync) { + protected void prepareOnAffectedNodes(TxInvocationContext ctx, PrepareCommand command, Collection
recipients) { if (log.isTraceEnabled()) { log.tracef("Total Order Anycast transaction %s with Total Order", command.getGlobalTransaction().globalId()); } @@ -67,11 +63,6 @@ protected void prepareOnAffectedNodes(TxInvocationContext ctx, PrepareCommand } } - @Override - protected void lockAndWrap(InvocationContext ctx, Object key, InternalCacheEntry ice, FlagAffectedCommand command) throws InterruptedException { - entryFactory.wrapEntryForPut(ctx, key, ice, false, command, true); - } - @Override protected Log getLog() { return log; diff --git a/core/src/main/java/org/infinispan/interceptors/totalorder/TotalOrderVersionedDistributionInterceptor.java b/core/src/main/java/org/infinispan/interceptors/totalorder/TotalOrderVersionedDistributionInterceptor.java index df41c1e54a3..3bf0b0f3850 100644 --- a/core/src/main/java/org/infinispan/interceptors/totalorder/TotalOrderVersionedDistributionInterceptor.java +++ b/core/src/main/java/org/infinispan/interceptors/totalorder/TotalOrderVersionedDistributionInterceptor.java @@ -1,14 +1,11 @@ package org.infinispan.interceptors.totalorder; -import org.infinispan.commands.FlagAffectedCommand; import org.infinispan.commands.tx.CommitCommand; import org.infinispan.commands.tx.PrepareCommand; import org.infinispan.commands.tx.RollbackCommand; import org.infinispan.commands.tx.VersionedPrepareCommand; import org.infinispan.commands.write.ClearCommand; import org.infinispan.configuration.cache.Configurations; -import org.infinispan.container.entries.InternalCacheEntry; -import org.infinispan.context.InvocationContext; import org.infinispan.context.impl.TxInvocationContext; import org.infinispan.interceptors.distribution.VersionedDistributionInterceptor; import org.infinispan.remoting.responses.KeysValidateFilter; @@ -50,7 +47,7 @@ public Object visitCommitCommand(TxInvocationContext ctx, CommitCommand command) } @Override - protected void prepareOnAffectedNodes(TxInvocationContext ctx, PrepareCommand command, Collection
recipients, boolean sync) { + protected void prepareOnAffectedNodes(TxInvocationContext ctx, PrepareCommand command, Collection
recipients) { if (log.isTraceEnabled()) { log.tracef("Total Order Anycast transaction %s with Total Order", command.getGlobalTransaction().globalId()); } @@ -77,11 +74,6 @@ protected void prepareOnAffectedNodes(TxInvocationContext ctx, PrepareCommand } } - @Override - protected void lockAndWrap(InvocationContext ctx, Object key, InternalCacheEntry ice, FlagAffectedCommand command) throws InterruptedException { - entryFactory.wrapEntryForPut(ctx, key, ice, false, command, true); - } - @Override protected Log getLog() { return log; diff --git a/core/src/main/java/org/infinispan/partitionhandling/impl/AvailablePartitionHandlingManager.java b/core/src/main/java/org/infinispan/partitionhandling/impl/AvailablePartitionHandlingManager.java new file mode 100644 index 00000000000..295e64e874d --- /dev/null +++ b/core/src/main/java/org/infinispan/partitionhandling/impl/AvailablePartitionHandlingManager.java @@ -0,0 +1,94 @@ +package org.infinispan.partitionhandling.impl; + +import org.infinispan.commands.write.WriteCommand; +import org.infinispan.container.versioning.EntryVersionsMap; +import org.infinispan.partitionhandling.AvailabilityMode; +import org.infinispan.remoting.transport.Address; +import org.infinispan.topology.CacheTopology; +import org.infinispan.transaction.xa.GlobalTransaction; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +/** + * {@link PartitionHandlingManager} implementation when the cluster is always available. + * + * @author Pedro Ruivo + * @since 8.0 + */ +public class AvailablePartitionHandlingManager implements PartitionHandlingManager { + + + private AvailablePartitionHandlingManager() { + } + + public static AvailablePartitionHandlingManager getInstance() { + return SingletonHolder.INSTANCE; + } + + @Override + public AvailabilityMode getAvailabilityMode() { + return AvailabilityMode.AVAILABLE; + } + + @Override + public void setAvailabilityMode(AvailabilityMode availabilityMode) {/*no-op*/} + + @Override + public void checkWrite(Object key) {/*no-op*/} + + @Override + public void checkRead(Object key) {/*no-op*/} + + @Override + public void checkClear() {/*no-op*/} + + @Override + public void checkBulkRead() {/*no-op*/} + + @Override + public CacheTopology getLastStableTopology() { + return null; + } + + @Override + public boolean addPartialRollbackTransaction(GlobalTransaction globalTransaction, Collection
affectedNodes, + Collection lockedKeys) { + return false; + } + + @Override + public boolean addPartialCommit2PCTransaction(GlobalTransaction globalTransaction, Collection
affectedNodes, + Collection lockedKeys, EntryVersionsMap newVersions) { + return false; + } + + @Override + public boolean addPartialCommit1PCTransaction(GlobalTransaction globalTransaction, Collection
affectedNodes, + Collection lockedKeys, List modifications) { + return false; + } + + @Override + public boolean isTransactionPartiallyCommitted(GlobalTransaction globalTransaction) { + return false; + } + + @Override + public Collection getPartialTransactions() { + return Collections.emptyList(); + } + + @Override + public boolean canRollbackTransactionAfterOriginatorLeave(GlobalTransaction globalTransaction) { + return true; + } + + @Override + public void onTopologyUpdate(CacheTopology cacheTopology) {/*no-op*/} + + private static class SingletonHolder { + private static final AvailablePartitionHandlingManager INSTANCE = new AvailablePartitionHandlingManager(); + } +} diff --git a/core/src/main/java/org/infinispan/partitionhandling/impl/PartitionHandlingInterceptor.java b/core/src/main/java/org/infinispan/partitionhandling/impl/PartitionHandlingInterceptor.java index 58a32ee72e6..4655550f9b5 100644 --- a/core/src/main/java/org/infinispan/partitionhandling/impl/PartitionHandlingInterceptor.java +++ b/core/src/main/java/org/infinispan/partitionhandling/impl/PartitionHandlingInterceptor.java @@ -161,7 +161,7 @@ public Object visitCommitCommand(TxInvocationContext ctx, CommitCommand command) } protected void postTxCommandCheck(TxInvocationContext ctx) { - if (ctx.hasModifications() && partitionHandlingManager.getAvailabilityMode() != AvailabilityMode.AVAILABLE) { + if (ctx.hasModifications() && partitionHandlingManager.getAvailabilityMode() != AvailabilityMode.AVAILABLE && !partitionHandlingManager.isTransactionPartiallyCommitted(ctx.getGlobalTransaction())) { for (Object key : ctx.getAffectedKeys()) { partitionHandlingManager.checkWrite(key); } diff --git a/core/src/main/java/org/infinispan/partitionhandling/impl/PartitionHandlingManager.java b/core/src/main/java/org/infinispan/partitionhandling/impl/PartitionHandlingManager.java index ee4e508c74d..a0367a27d7c 100644 --- a/core/src/main/java/org/infinispan/partitionhandling/impl/PartitionHandlingManager.java +++ b/core/src/main/java/org/infinispan/partitionhandling/impl/PartitionHandlingManager.java @@ -1,17 +1,24 @@ package org.infinispan.partitionhandling.impl; +import org.infinispan.commands.write.WriteCommand; +import org.infinispan.container.versioning.EntryVersionsMap; import org.infinispan.partitionhandling.AvailabilityMode; +import org.infinispan.remoting.transport.Address; import org.infinispan.topology.CacheTopology; +import org.infinispan.transaction.xa.GlobalTransaction; + +import java.util.Collection; +import java.util.List; /** * @author Dan Berindei * @since 7.0 */ public interface PartitionHandlingManager { - void setAvailabilityMode(AvailabilityMode availabilityMode); - AvailabilityMode getAvailabilityMode(); + void setAvailabilityMode(AvailabilityMode availabilityMode); + void checkWrite(Object key); void checkRead(Object key); @@ -20,5 +27,86 @@ public interface PartitionHandlingManager { void checkBulkRead(); + @Deprecated //test use only. it can be removed if we update the tests CacheTopology getLastStableTopology(); + + /** + * Adds a partially aborted transaction. + *

+ * The transaction should be registered when it is not sure if the abort happens successfully in all the affected + * nodes. + * + * @param globalTransaction the global transaction. + * @param affectedNodes the nodes involved in the transaction and they must abort the transaction. + * @param lockedKeys the keys locally locked. + * @return {@code true} if the {@link PartitionHandlingManager} will handle it, {@code false} otherwise. + */ + boolean addPartialRollbackTransaction(GlobalTransaction globalTransaction, Collection

affectedNodes, + Collection lockedKeys); + + /** + * Adds a partially committed transaction. + *

+ * The transaction is committed in the second phase and it is register if it is not sure that the transaction was + * committed successfully in all the affected nodes. + * + * @param globalTransaction the global transaction. + * @param affectedNodes the nodes involved in the transaction and they must commit it. + * @param lockedKeys the keys locally locked. + * @param newVersions the updated versions. Only used when versioning is enabled. + * @return {@code true} if the {@link PartitionHandlingManager} will handle it, {@code false} otherwise. + */ + boolean addPartialCommit2PCTransaction(GlobalTransaction globalTransaction, Collection

affectedNodes, + Collection lockedKeys, EntryVersionsMap newVersions); + + /** + * Adds a partially committed transaction. + *

+ * The transaction is committed in one phase and it is register if it is not sure that the transaction was committed + * successfully in all the affected nodes. + * + * @param globalTransaction the global transaction. + * @param affectedNodes the nodes involved in the transaction and they must commit it. + * @param lockedKeys the keys locally locked. + * @param modifications the transaction's modification log. + * @return {@code true} if the {@link PartitionHandlingManager} will handle it, {@code false} otherwise. + */ + boolean addPartialCommit1PCTransaction(GlobalTransaction globalTransaction, Collection

affectedNodes, + Collection lockedKeys, List modifications); + + /** + * It checks if the transaction resources (for example locks) can be released. + *

+ * The transaction resource can't be released when the transaction is partially committed. + * + * @param globalTransaction the transaction. + * @return {@code true} if the resources can be released, {@code false} otherwise. + */ + boolean isTransactionPartiallyCommitted(GlobalTransaction globalTransaction); + + /** + * @return a collection of partial committed or aborted transactions. + */ + Collection getPartialTransactions(); + + /** + * It checks if the transaction can be aborted when the originator leaves the cluster. + *

+ * The only case in which it is not possible to abort is when partition handling is enabled and the originator didn't + * leave gracefully. The transaction will complete when the partition heals. + * + * @param globalTransaction the global transaction. + * @return {@code true} if the transaction can be aborted, {@code false} otherwise. + */ + boolean canRollbackTransactionAfterOriginatorLeave(GlobalTransaction globalTransaction); + + /** + * Notifies the {@link PartitionHandlingManager} that the cache topology was update. + *

+ * It detects when the partition is merged and tries to complete all the partially completed transactions. + * + * @param cacheTopology the new cache topology. + */ + void onTopologyUpdate(CacheTopology cacheTopology); + } diff --git a/core/src/main/java/org/infinispan/partitionhandling/impl/PartitionHandlingManagerImpl.java b/core/src/main/java/org/infinispan/partitionhandling/impl/PartitionHandlingManagerImpl.java index e545127d021..73910425c73 100644 --- a/core/src/main/java/org/infinispan/partitionhandling/impl/PartitionHandlingManagerImpl.java +++ b/core/src/main/java/org/infinispan/partitionhandling/impl/PartitionHandlingManagerImpl.java @@ -1,24 +1,40 @@ package org.infinispan.partitionhandling.impl; import org.infinispan.Cache; +import org.infinispan.commands.CommandsFactory; +import org.infinispan.commands.ReplicableCommand; +import org.infinispan.commands.remote.recovery.TxCompletionNotificationCommand; +import org.infinispan.commands.tx.VersionedCommitCommand; +import org.infinispan.commands.write.WriteCommand; +import org.infinispan.commons.util.CollectionFactory; +import org.infinispan.configuration.cache.Configuration; +import org.infinispan.configuration.cache.Configurations; +import org.infinispan.container.versioning.EntryVersionsMap; import org.infinispan.distribution.DistributionManager; import org.infinispan.factories.annotations.Inject; import org.infinispan.factories.annotations.Start; import org.infinispan.notifications.cachelistener.CacheNotifier; import org.infinispan.partitionhandling.AvailabilityMode; +import org.infinispan.remoting.inboundhandler.DeliverOrder; +import org.infinispan.remoting.responses.CacheNotFoundResponse; +import org.infinispan.remoting.responses.Response; +import org.infinispan.remoting.responses.UnsureResponse; +import org.infinispan.remoting.rpc.RpcManager; import org.infinispan.remoting.transport.Address; import org.infinispan.statetransfer.StateTransferManager; import org.infinispan.topology.CacheTopology; import org.infinispan.topology.LocalTopologyManager; +import org.infinispan.transaction.xa.GlobalTransaction; +import org.infinispan.util.concurrent.locks.LockManager; import org.infinispan.util.logging.Log; import org.infinispan.util.logging.LogFactory; -import java.util.List; +import java.util.*; public class PartitionHandlingManagerImpl implements PartitionHandlingManager { private static final Log log = LogFactory.getLog(PartitionHandlingManagerImpl.class); private static final boolean trace = log.isTraceEnabled(); - + private final Map partialTransactions; private volatile AvailabilityMode availabilityMode = AvailabilityMode.AVAILABLE; private DistributionManager distributionManager; @@ -26,19 +42,40 @@ public class PartitionHandlingManagerImpl implements PartitionHandlingManager { private StateTransferManager stateTransferManager; private String cacheName; private CacheNotifier notifier; + private CommandsFactory commandsFactory; + private Configuration configuration; + private RpcManager rpcManager; + private LockManager lockManager; + + private boolean isVersioned; + + public PartitionHandlingManagerImpl() { + partialTransactions = CollectionFactory.makeConcurrentMap(); + } @Inject - void init(DistributionManager distributionManager, LocalTopologyManager localTopologyManager, - StateTransferManager stateTransferManager, Cache cache, CacheNotifier notifier) { + public void init(DistributionManager distributionManager, LocalTopologyManager localTopologyManager, + StateTransferManager stateTransferManager, Cache cache, CacheNotifier notifier, CommandsFactory commandsFactory, + Configuration configuration, RpcManager rpcManager, LockManager lockManager) { this.distributionManager = distributionManager; this.localTopologyManager = localTopologyManager; this.stateTransferManager = stateTransferManager; this.cacheName = cache.getName(); this.notifier = notifier; + this.commandsFactory = commandsFactory; + this.configuration = configuration; + this.rpcManager = rpcManager; + this.lockManager = lockManager; } @Start - void start() { + public void start() { + isVersioned = Configurations.isVersioningEnabled(configuration); + } + + @Override + public AvailabilityMode getAvailabilityMode() { + return availabilityMode; } @Override @@ -51,11 +88,6 @@ public void setAvailabilityMode(AvailabilityMode availabilityMode) { } } - @Override - public AvailabilityMode getAvailabilityMode() { - return availabilityMode; - } - @Override public void checkWrite(Object key) { doCheck(key); @@ -66,6 +98,146 @@ public void checkRead(Object key) { doCheck(key); } + @Override + public void checkClear() { + if (availabilityMode != AvailabilityMode.AVAILABLE) { + throw log.clearDisallowedWhilePartitioned(); + } + } + + @Override + public void checkBulkRead() { + if (availabilityMode != AvailabilityMode.AVAILABLE) { + throw log.partitionDegraded(); + } + } + + @Override + public CacheTopology getLastStableTopology() { + return localTopologyManager.getStableCacheTopology(cacheName); + } + + @Override + public boolean addPartialRollbackTransaction(GlobalTransaction globalTransaction, Collection

affectedNodes, Collection lockedKeys) { + if (trace) { + log.tracef("Added partially rollback transaction %s", globalTransaction); + } + partialTransactions.put(globalTransaction, new RollbackTransactionInfo(globalTransaction, affectedNodes, lockedKeys)); + return true; + } + + @Override + public boolean addPartialCommit2PCTransaction(GlobalTransaction globalTransaction, Collection
affectedNodes, + Collection lockedKeys, EntryVersionsMap newVersions) { + if (trace) { + log.tracef("Added partially committed (2PC) transaction %s", globalTransaction); + } + partialTransactions.put(globalTransaction, new Commit2PCTransactionInfo(globalTransaction, affectedNodes, + lockedKeys, newVersions)); + return true; + } + + @Override + public boolean addPartialCommit1PCTransaction(GlobalTransaction globalTransaction, Collection
affectedNodes, + Collection lockedKeys, List modifications) { + if (trace) { + log.tracef("Added partially committed (1PC) transaction %s", globalTransaction); + } + partialTransactions.put(globalTransaction, new Commit1PCTransactionInfo(globalTransaction, affectedNodes, + lockedKeys, modifications)); + return true; + } + + @Override + public boolean isTransactionPartiallyCommitted(GlobalTransaction globalTransaction) { + TransactionInfo transactionInfo = partialTransactions.get(globalTransaction); + if (trace) { + log.tracef("Can release resources for transaction %s. Transaction info=%s", globalTransaction, transactionInfo); + } + return transactionInfo != null && !transactionInfo.isRolledBack(); //if we are going to commit, we can't release the resources yet + } + + @Override + public Collection getPartialTransactions() { + return Collections.unmodifiableCollection(partialTransactions.keySet()); + } + + @Override + public boolean canRollbackTransactionAfterOriginatorLeave(GlobalTransaction globalTransaction) { + boolean canRollback = availabilityMode == AvailabilityMode.AVAILABLE && + !getLastStableTopology().getActualMembers().contains(globalTransaction.getAddress()); + if (trace) { + log.tracef("Can rollback transaction? %s", canRollback); + } + return canRollback; + } + + @Override + public void onTopologyUpdate(CacheTopology cacheTopology) { + boolean isStable = isTopologyStable(cacheTopology); + if (trace) { + log.tracef("On stable topology update. Has pending tx? %b. Is stable? %b. topology=%s", + !partialTransactions.isEmpty(), isStable, cacheTopology); + } + if (isStable) { + if (!partialTransactions.isEmpty()) { + for (TransactionInfo transactionInfo : partialTransactions.values()) { + completeTransaction(transactionInfo, cacheTopology); + } + } + } + } + + private void completeTransaction(final TransactionInfo transactionInfo, CacheTopology cacheTopology) { + rpcManager.invokeRemotelyAsync(transactionInfo.getCommitNodes(cacheTopology), + transactionInfo.buildCommand(commandsFactory, isVersioned), + rpcManager.getDefaultRpcOptions(true)) + .whenComplete((responseMap, throwable) -> { + final GlobalTransaction globalTransaction = transactionInfo.getGlobalTransaction(); + if (trace) { + log.tracef("Future done for transaction %s", globalTransaction); + } + + if (throwable != null) { + if (trace) { + log.tracef(throwable, "Exception for transaction %s. Retry later.", globalTransaction); + } + return; + } + + if (trace) { + log.tracef("Future done for transaction %s. Response are %s", globalTransaction, responseMap); + } + + for (Response response : responseMap.values()) { + if (response == UnsureResponse.INSTANCE || response == CacheNotFoundResponse.INSTANCE) { + if (trace) { + log.tracef("Another partition or topology changed for transaction %s. Retry later.", globalTransaction); + } + return; + } + } + if (trace) { + log.tracef("Performing cleanup for transaction %s", globalTransaction); + } + lockManager.unlock(transactionInfo.getLockedKeys(), globalTransaction); + partialTransactions.remove(globalTransaction); + TxCompletionNotificationCommand command = commandsFactory.buildTxCompletionNotificationCommand(null, globalTransaction); + //a little bit overkill, but the state transfer can happen during a merge and some nodes can receive the + // transaction that aren't in the original affected nodes. + //no side effects. + rpcManager.invokeRemotely(null, command, rpcManager.getDefaultRpcOptions(false, DeliverOrder.NONE)); + }); + } + + private boolean isTopologyStable(CacheTopology cacheTopology) { + CacheTopology stableTopology = localTopologyManager.getStableCacheTopology(cacheName); + if (trace) { + log.tracef("Check if topology %s is stable. Last stable topology is %s", cacheTopology, stableTopology); + } + return stableTopology != null && cacheTopology.getActualMembers().containsAll(stableTopology.getActualMembers()); + } + private void doCheck(Object key) { if (trace) log.tracef("Checking availability for key=%s, status=%s", key, availabilityMode); if (availabilityMode == AvailabilityMode.AVAILABLE) @@ -81,22 +253,120 @@ private void doCheck(Object key) { } } - @Override - public void checkClear() { - if (availabilityMode != AvailabilityMode.AVAILABLE) { - throw log.clearDisallowedWhilePartitioned(); + private interface TransactionInfo { + boolean isRolledBack(); + + List
getCommitNodes(CacheTopology stableTopology); + + ReplicableCommand buildCommand(CommandsFactory commandsFactory, boolean isVersioned); + + GlobalTransaction getGlobalTransaction(); + + Collection getLockedKeys(); + } + + private static class RollbackTransactionInfo extends BaseTransactionInfo { + + protected RollbackTransactionInfo(GlobalTransaction globalTransaction, Collection
affectedNodes, Collection lockedKeys) { + super(globalTransaction, affectedNodes, lockedKeys); + } + + @Override + public boolean isRolledBack() { + return true; + } + + @Override + public ReplicableCommand buildCommand(CommandsFactory commandsFactory, boolean isVersioned) { + return commandsFactory.buildRollbackCommand(getGlobalTransaction()); } + } - @Override - public void checkBulkRead() { - if (availabilityMode != AvailabilityMode.AVAILABLE) { - throw log.partitionDegraded(); + private static class Commit2PCTransactionInfo extends BaseTransactionInfo { + + private final EntryVersionsMap newVersions; + + public Commit2PCTransactionInfo(GlobalTransaction globalTransaction, Collection
affectedNodes, Collection lockedKeys, EntryVersionsMap newVersions) { + super(globalTransaction, affectedNodes, lockedKeys); + this.newVersions = newVersions; + } + + @Override + public boolean isRolledBack() { + return false; + } + + @Override + public ReplicableCommand buildCommand(CommandsFactory commandsFactory, boolean isVersioned) { + if (isVersioned) { + VersionedCommitCommand commitCommand = commandsFactory.buildVersionedCommitCommand(getGlobalTransaction()); + commitCommand.setUpdatedVersions(newVersions); + return commitCommand; + } else { + return commandsFactory.buildCommitCommand(getGlobalTransaction()); + } } } - @Override - public CacheTopology getLastStableTopology() { - return localTopologyManager.getStableCacheTopology(cacheName); + private static class Commit1PCTransactionInfo extends BaseTransactionInfo { + + private final List modifications; + + public Commit1PCTransactionInfo(GlobalTransaction globalTransaction, Collection
affectedNodes, Collection lockedKeys, List modifications) { + super(globalTransaction, affectedNodes, lockedKeys); + this.modifications = modifications; + } + + @Override + public boolean isRolledBack() { + return false; + } + + @Override + public ReplicableCommand buildCommand(CommandsFactory commandsFactory, boolean isVersioned) { + if (isVersioned) { + throw new IllegalArgumentException("Cannot build a versioned one-phase-commit prepare command."); + } + return commandsFactory.buildPrepareCommand(getGlobalTransaction(), modifications, true); + } + } + + private static abstract class BaseTransactionInfo implements TransactionInfo { + private final GlobalTransaction globalTransaction; + private final List
affectedNodes; + private final Collection lockedKeys; + + protected BaseTransactionInfo(GlobalTransaction globalTransaction, Collection
affectedNodes, Collection lockedKeys) { + this.globalTransaction = globalTransaction; + this.lockedKeys = lockedKeys; + this.affectedNodes = new ArrayList<>(affectedNodes); + } + + @Override + public final List
getCommitNodes(CacheTopology stableTopology) { + List
commitNodes = new ArrayList<>(affectedNodes); + commitNodes.retainAll(stableTopology.getActualMembers()); + return commitNodes; + } + + @Override + public final GlobalTransaction getGlobalTransaction() { + return globalTransaction; + } + + @Override + public Collection getLockedKeys() { + return lockedKeys; + } + + @Override + public String toString() { + return "TransactionInfo{" + + "globalTransaction=" + globalTransaction + ", " + + "rollback=" + isRolledBack() + ", " + + "affectedNodes=" + affectedNodes + + '}'; + } } } diff --git a/core/src/main/java/org/infinispan/remoting/transport/AbstractDelegatingTransport.java b/core/src/main/java/org/infinispan/remoting/transport/AbstractDelegatingTransport.java index 1094436f292..f781ced1d08 100644 --- a/core/src/main/java/org/infinispan/remoting/transport/AbstractDelegatingTransport.java +++ b/core/src/main/java/org/infinispan/remoting/transport/AbstractDelegatingTransport.java @@ -123,6 +123,10 @@ public Log getLog() { return actual.getLog(); } + public Transport getDelegate() { + return actual; + } + /** * method invoked before a remote invocation. * diff --git a/core/src/main/java/org/infinispan/remoting/transport/jgroups/JGroupsTransport.java b/core/src/main/java/org/infinispan/remoting/transport/jgroups/JGroupsTransport.java index 03b475ef7bd..0264aa25a56 100644 --- a/core/src/main/java/org/infinispan/remoting/transport/jgroups/JGroupsTransport.java +++ b/core/src/main/java/org/infinispan/remoting/transport/jgroups/JGroupsTransport.java @@ -612,6 +612,15 @@ public CompletableFuture> invokeRemotelyAsync(Collection< throw new TimeoutException("Timed out waiting for valid responses!"); } } + + if (recipients != null) { + for (Address dest : recipients) { + if (!dest.equals(getAddress()) && !retval.containsKey(dest)) { + retval.put(dest, CacheNotFoundResponse.INSTANCE); + } + } + } + return retval; }); } else { diff --git a/core/src/main/java/org/infinispan/statetransfer/StateConsumerImpl.java b/core/src/main/java/org/infinispan/statetransfer/StateConsumerImpl.java index 8fd68a90b91..a19eb96b812 100644 --- a/core/src/main/java/org/infinispan/statetransfer/StateConsumerImpl.java +++ b/core/src/main/java/org/infinispan/statetransfer/StateConsumerImpl.java @@ -289,7 +289,6 @@ public void onTopologyUpdate(final CacheTopology cacheTopology, final boolean is // even if some of the tasks are removed and re-added waitingForState.set(false); - final ConsistentHash newReadCh = cacheTopology.getReadConsistentHash(); final ConsistentHash newWriteCh = cacheTopology.getWriteConsistentHash(); final ConsistentHash previousReadCh = this.cacheTopology != null ? this.cacheTopology.getReadConsistentHash() : null; final ConsistentHash previousWriteCh = this.cacheTopology != null ? this.cacheTopology.getWriteConsistentHash() : null; @@ -521,17 +520,16 @@ public void applyState(final Address sender, int topologyId, Collection mySegments = wCh.getSegmentsForOwner(rpcManager.getAddress()); final CountDownLatch countDownLatch = new CountDownLatch(stateChunks.size()); for (final StateChunk stateChunk : stateChunks) { - stateTransferExecutor.submit(new Callable() { + stateTransferExecutor.submit(new Runnable() { @Override - public Void call() throws Exception { + public void run() { applyChunk(sender, mySegments, stateChunk); countDownLatch.countDown(); - return null; } }); } @@ -547,7 +545,7 @@ public Void call() throws Exception { if (trace) { log.tracef("After applying the received state the data container of cache %s has %d keys", cacheName, - dataContainer.size()); + dataContainer.size()); synchronized (transferMapsLock) { log.tracef("Segments not received yet for cache %s: %s", cacheName, transfersBySource); } @@ -630,6 +628,9 @@ private void applyTransactions(Address sender, Collection trans if (isTransactional) { for (TransactionInfo transactionInfo : transactions) { GlobalTransaction gtx = transactionInfo.getGlobalTransaction(); + if (rpcManager.getAddress().equals(gtx.getAddress())) { + continue; // it is a transaction originated in this node. can happen with partition handling + } // Mark the global transaction as remote. Only used for logging, hashCode/equals ignore it. gtx.setRemote(true); @@ -917,8 +918,6 @@ private void removeStaleData(final Set removedSegments) throws Interrup if (removedSegments.isEmpty()) return; - // Keys that might have been in L1, and need to be removed from the data container - final ConcurrentHashSet keysToInvalidate = new ConcurrentHashSet(); // Keys that we used to own, and need to be removed from the data container AND the cache stores final ConcurrentHashSet keysToRemove = new ConcurrentHashSet(); @@ -973,15 +972,16 @@ public void processEntry(MarshalledEntry marshalledEntry, AdvancedCacheLoader.Ta * @param addedSegments */ private void restartBrokenTransfers(CacheTopology cacheTopology, Set addedSegments) { - Set
members = new HashSet
(cacheTopology.getReadConsistentHash().getMembers()); + Set
members = new HashSet<>(cacheTopology.getReadConsistentHash().getMembers()); synchronized (transferMapsLock) { - for (Iterator
it = transfersBySource.keySet().iterator(); it.hasNext(); ) { - Address source = it.next(); + for (Iterator>> it = transfersBySource.entrySet().iterator(); it.hasNext(); ) { + Map.Entry> entry = it.next(); + Address source = entry.getKey(); if (!members.contains(source)) { if (trace) { log.tracef("Removing inbound transfers from source %s for cache %s", source, cacheName); } - List inboundTransfers = transfersBySource.get(source); + List inboundTransfers = entry.getValue(); it.remove(); for (InboundTransferTask inboundTransfer : inboundTransfers) { // these segments will be restarted if they are still in new write CH diff --git a/core/src/main/java/org/infinispan/statetransfer/StateTransferInterceptor.java b/core/src/main/java/org/infinispan/statetransfer/StateTransferInterceptor.java index 75f1195f38a..3065084a537 100644 --- a/core/src/main/java/org/infinispan/statetransfer/StateTransferInterceptor.java +++ b/core/src/main/java/org/infinispan/statetransfer/StateTransferInterceptor.java @@ -24,7 +24,6 @@ import org.infinispan.context.Flag; import org.infinispan.context.InvocationContext; import org.infinispan.context.impl.TxInvocationContext; -import org.infinispan.factories.ComponentRegistry; import org.infinispan.factories.annotations.Inject; import org.infinispan.interceptors.base.BaseStateTransferInterceptor; import org.infinispan.remoting.RemoteException; @@ -66,7 +65,6 @@ public class StateTransferInterceptor extends BaseStateTransferInterceptor { private StateTransferManager stateTransferManager; private Transport transport; - private ComponentRegistry componentRegistry; private final AffectedKeysVisitor affectedKeysVisitor = new AffectedKeysVisitor(); @@ -76,11 +74,9 @@ protected Log getLog() { } @Inject - public void init(StateTransferManager stateTransferManager, Transport transport, - ComponentRegistry componentRegistry) { + public void init(StateTransferManager stateTransferManager, Transport transport) { this.stateTransferManager = stateTransferManager; this.transport = transport; - this.componentRegistry = componentRegistry; } @Override @@ -152,7 +148,7 @@ public Object visitEvictCommand(InvocationContext ctx, EvictCommand command) thr @Override public Object visitGetAllCommand(InvocationContext ctx, GetAllCommand command) throws Throwable { - if (isLocalOnly(ctx, command)) { + if (isLocalOnly(command)) { return invokeNextInterceptor(ctx, command); } CacheTopology beginTopology = stateTransferManager.getCacheTopology(); @@ -189,7 +185,7 @@ private Object handleTxCommand(TxInvocationContext ctx, TransactionBoundaryComma Address origin = ctx.isOriginLocal() ? ctx.getOrigin() : ctx.getGlobalTransaction().getAddress(); if (trace) log.tracef("handleTxCommand for command %s, origin %s", command, origin); - if (isLocalOnly(ctx, command)) { + if (isLocalOnly(command)) { return invokeNextInterceptor(ctx, command); } updateTopologyId(command); @@ -206,7 +202,7 @@ private Object handleTxCommand(TxInvocationContext ctx, TransactionBoundaryComma // We need to forward the command to the new owners, if the command was asynchronous boolean async = isTxCommandAsync(command); if (async) { - stateTransferManager.forwardCommandIfNeeded(command, getAffectedKeys(ctx, command), origin, false); + stateTransferManager.forwardCommandIfNeeded(command, getAffectedKeys(ctx, command), origin); return null; } @@ -254,7 +250,7 @@ private Object handleTxWriteCommand(InvocationContext ctx, WriteCommand command) Address origin = ctx.getOrigin(); if (trace) log.tracef("handleTxWriteCommand for command %s, origin %s", command, origin); - if (isLocalOnly(ctx, command)) { + if (isLocalOnly(command)) { return invokeNextInterceptor(ctx, command); } updateTopologyId(command); @@ -300,7 +296,7 @@ private Object handleTxWriteCommand(InvocationContext ctx, WriteCommand command) private Object handleNonTxWriteCommand(InvocationContext ctx, WriteCommand command) throws Throwable { if (trace) log.tracef("handleNonTxWriteCommand for command %s, topology id %d", command, command.getTopologyId()); - if (isLocalOnly(ctx, command)) { + if (isLocalOnly(command)) { return invokeNextInterceptor(ctx, command); } @@ -356,7 +352,7 @@ private Object handleTopologyAffectedCommand(InvocationContext ctx, VisitableCom Address origin) throws Throwable { if (trace) log.tracef("handleTopologyAffectedCommand for command %s, origin %s", command, origin); - if (isLocalOnly(ctx, command)) { + if (isLocalOnly(command)) { return invokeNextInterceptor(ctx, command); } updateTopologyId((TopologyAffectedCommand) command); @@ -364,7 +360,7 @@ private Object handleTopologyAffectedCommand(InvocationContext ctx, VisitableCom return invokeNextInterceptor(ctx, command); } - private boolean isLocalOnly(InvocationContext ctx, VisitableCommand command) { + private boolean isLocalOnly(VisitableCommand command) { boolean cacheModeLocal = false; if (command instanceof FlagAffectedCommand) { cacheModeLocal = ((FlagAffectedCommand)command).hasFlag(Flag.CACHE_MODE_LOCAL); diff --git a/core/src/main/java/org/infinispan/statetransfer/StateTransferManager.java b/core/src/main/java/org/infinispan/statetransfer/StateTransferManager.java index bcebc60e91f..19cdb11855b 100644 --- a/core/src/main/java/org/infinispan/statetransfer/StateTransferManager.java +++ b/core/src/main/java/org/infinispan/statetransfer/StateTransferManager.java @@ -53,7 +53,7 @@ public interface StateTransferManager { * If there is an state transfer happening at the moment, this method forwards the supplied * command to the nodes that are new owners of the data, in order to assure consistency. */ - Map forwardCommandIfNeeded(TopologyAffectedCommand command, Set affectedKeys, Address origin, boolean sync); + Map forwardCommandIfNeeded(TopologyAffectedCommand command, Set affectedKeys, Address origin); void notifyEndOfRebalance(int topologyId, int rebalanceId); diff --git a/core/src/main/java/org/infinispan/statetransfer/StateTransferManagerImpl.java b/core/src/main/java/org/infinispan/statetransfer/StateTransferManagerImpl.java index 639bc32a6f4..d0260cf6d33 100644 --- a/core/src/main/java/org/infinispan/statetransfer/StateTransferManagerImpl.java +++ b/core/src/main/java/org/infinispan/statetransfer/StateTransferManagerImpl.java @@ -20,7 +20,9 @@ import org.infinispan.partitionhandling.impl.PartitionHandlingManager; import org.infinispan.remoting.inboundhandler.DeliverOrder; import org.infinispan.remoting.responses.Response; +import org.infinispan.remoting.rpc.ResponseMode; import org.infinispan.remoting.rpc.RpcManager; +import org.infinispan.remoting.rpc.RpcOptions; import org.infinispan.remoting.transport.Address; import org.infinispan.topology.CacheJoinInfo; import org.infinispan.topology.CacheTopology; @@ -207,6 +209,7 @@ private void doTopologyUpdate(CacheTopology newCacheTopology, boolean isRebalanc log.tracef("Initial state transfer complete for cache %s on node %s", cacheName, rpcManager.getAddress()); } } + partitionHandlingManager.onTopologyUpdate(newCacheTopology); } @Start(priority = 1000) @@ -257,12 +260,18 @@ public CacheTopology getCacheTopology() { @Override public Map forwardCommandIfNeeded(TopologyAffectedCommand command, Set affectedKeys, - Address origin, boolean sync) { + Address origin) { + final CacheTopology cacheTopology = getCacheTopology(); + if (cacheTopology == null) { + if (trace) { + log.tracef("Not fowarding command %s because topology is null.", command); + } + return Collections.emptyMap(); + } int cmdTopologyId = command.getTopologyId(); // forward commands with older topology ids to their new targets // but we need to make sure we have the latest topology - CacheTopology cacheTopology = getCacheTopology(); - int localTopologyId = cacheTopology != null ? cacheTopology.getTopologyId() : -1; + int localTopologyId = cacheTopology.getTopologyId(); // if it's a tx/lock/write command, forward it to the new owners if (trace) { log.tracef("CommandTopologyId=%s, localTopologyId=%s", cmdTopologyId, localTopologyId); @@ -270,7 +279,7 @@ public Map forwardCommandIfNeeded(TopologyAffectedCommand com if (cmdTopologyId < localTopologyId) { ConsistentHash writeCh = cacheTopology.getWriteConsistentHash(); - Set
newTargets = new HashSet
(writeCh.locateAllOwners(affectedKeys)); + Set
newTargets = new HashSet<>(writeCh.locateAllOwners(affectedKeys)); newTargets.remove(rpcManager.getAddress()); // Forwarding to the originator would create a cycle // TODO This may not be the "real" originator, but one of the original recipients @@ -284,9 +293,10 @@ public Map forwardCommandIfNeeded(TopologyAffectedCommand com if (trace) { log.tracef("Forwarding command %s to new targets %s", command, newTargets); } + final RpcOptions rpcOptions = rpcManager.getDefaultRpcOptions(false, DeliverOrder.NONE); // TODO find a way to forward the command async if it was received async // TxCompletionNotificationCommands are the only commands forwarded asynchronously, and they must be OOB - return rpcManager.invokeRemotely(newTargets, command, rpcManager.getDefaultRpcOptions(sync, DeliverOrder.NONE)); + return rpcManager.invokeRemotely(newTargets, command, rpcOptions); } } return Collections.emptyMap(); diff --git a/core/src/main/java/org/infinispan/topology/LocalTopologyManagerImpl.java b/core/src/main/java/org/infinispan/topology/LocalTopologyManagerImpl.java index c71c6a2cea8..cc2f476adf5 100644 --- a/core/src/main/java/org/infinispan/topology/LocalTopologyManagerImpl.java +++ b/core/src/main/java/org/infinispan/topology/LocalTopologyManagerImpl.java @@ -174,11 +174,9 @@ public ManagerStatusResponse handleStatusRequest(int viewId) { for (Map.Entry e : runningCaches.entrySet()) { String cacheName = e.getKey(); LocalCacheStatus cacheStatus = runningCaches.get(cacheName); - AvailabilityMode availabilityMode = cacheStatus.getPartitionHandlingManager() != null ? - cacheStatus.getPartitionHandlingManager().getAvailabilityMode() : null; caches.put(e.getKey(), new CacheStatusResponse(cacheStatus.getJoinInfo(), cacheStatus.getCurrentTopology(), cacheStatus.getStableTopology(), - availabilityMode)); + cacheStatus.getPartitionHandlingManager().getAvailabilityMode())); } } @@ -272,10 +270,8 @@ protected void doHandleTopologyUpdate(String cacheName, CacheTopology cacheTopol unionTopology.logRoutingTableInformation(); boolean updateAvailabilityModeFirst = availabilityMode != AvailabilityMode.AVAILABLE; - if (updateAvailabilityModeFirst) { - if (cacheStatus.getPartitionHandlingManager() != null && availabilityMode != null) { - cacheStatus.getPartitionHandlingManager().setAvailabilityMode(availabilityMode); - } + if (updateAvailabilityModeFirst && availabilityMode != null) { + cacheStatus.getPartitionHandlingManager().setAvailabilityMode(availabilityMode); } if ((existingTopology == null || existingTopology.getRebalanceId() != cacheTopology.getRebalanceId()) && unionCH != null) { // This CH_UPDATE command was sent after a REBALANCE_START command, but arrived first. @@ -287,9 +283,7 @@ protected void doHandleTopologyUpdate(String cacheName, CacheTopology cacheTopol } if (!updateAvailabilityModeFirst) { - if (cacheStatus.getPartitionHandlingManager() != null && availabilityMode != null) { - cacheStatus.getPartitionHandlingManager().setAvailabilityMode(availabilityMode); - } + cacheStatus.getPartitionHandlingManager().setAvailabilityMode(availabilityMode); } } } @@ -425,13 +419,13 @@ protected void doHandleRebalance(int viewId, LocalCacheStatus cacheStatus, Cache @Override public CacheTopology getCacheTopology(String cacheName) { LocalCacheStatus cacheStatus = runningCaches.get(cacheName); - return cacheStatus.getCurrentTopology(); + return cacheStatus != null ? cacheStatus.getCurrentTopology() : null; } @Override public CacheTopology getStableCacheTopology(String cacheName) { LocalCacheStatus cacheStatus = runningCaches.get(cacheName); - return cacheStatus.getCurrentTopology(); + return cacheStatus != null ? cacheStatus.getStableTopology() : null; } @Override @@ -487,10 +481,8 @@ public String getClusterAvailability() { AvailabilityMode clusterAvailability = AvailabilityMode.AVAILABLE; synchronized (runningCaches) { for (LocalCacheStatus cacheStatus : runningCaches.values()) { - AvailabilityMode availabilityMode = cacheStatus.getPartitionHandlingManager() != null ? - cacheStatus.getPartitionHandlingManager().getAvailabilityMode() : null; - clusterAvailability = availabilityMode != null ? clusterAvailability.min(availabilityMode) : clusterAvailability; - + AvailabilityMode availabilityMode = cacheStatus.getPartitionHandlingManager().getAvailabilityMode(); + clusterAvailability = clusterAvailability.min(availabilityMode); } } return clusterAvailability.toString(); @@ -499,9 +491,7 @@ public String getClusterAvailability() { @Override public AvailabilityMode getCacheAvailability(String cacheName) { LocalCacheStatus cacheStatus = runningCaches.get(cacheName); - AvailabilityMode availabilityMode = cacheStatus.getPartitionHandlingManager() != null ? - cacheStatus.getPartitionHandlingManager().getAvailabilityMode() : AvailabilityMode.AVAILABLE; - return availabilityMode; + return cacheStatus.getPartitionHandlingManager().getAvailabilityMode(); } @Override @@ -540,16 +530,15 @@ private Object executeOnCoordinator(ReplicableCommand command, long timeout) thr private void executeOnCoordinatorAsync(final ReplicableCommand command) throws Exception { // if we are the coordinator, the execution is actually synchronous if (transport.isCoordinator()) { - asyncTransportExecutor.submit(new Callable() { + asyncTransportExecutor.execute(new Runnable() { @Override - public Object call() throws Exception { + public void run() { if (log.isTraceEnabled()) log.tracef("Attempting to execute command on self: %s", command); gcr.wireDependencies(command); try { - return command.perform(null); + command.perform(null); } catch (Throwable t) { log.errorf(t, "Failed to execute ReplicableCommand %s on coordinator async: %s", command, t.getMessage()); - throw new Exception(t); } } }); @@ -673,6 +662,7 @@ public CacheTopology getStableTopology() { public void setStableTopology(CacheTopology stableTopology) { this.stableTopology = stableTopology; + partitionHandlingManager.onTopologyUpdate(currentTopology); } public SemaphoreCompletionService getTopologyUpdatesCompletionService() { diff --git a/core/src/main/java/org/infinispan/transaction/impl/AbstractEnlistmentAdapter.java b/core/src/main/java/org/infinispan/transaction/impl/AbstractEnlistmentAdapter.java index d4ab79418d0..ae90d65c3d7 100644 --- a/core/src/main/java/org/infinispan/transaction/impl/AbstractEnlistmentAdapter.java +++ b/core/src/main/java/org/infinispan/transaction/impl/AbstractEnlistmentAdapter.java @@ -5,6 +5,7 @@ import org.infinispan.configuration.cache.Configuration; import org.infinispan.configuration.cache.Configurations; import org.infinispan.interceptors.locking.ClusteringDependentLogic; +import org.infinispan.partitionhandling.impl.PartitionHandlingManager; import org.infinispan.remoting.inboundhandler.DeliverOrder; import org.infinispan.remoting.rpc.RpcManager; import org.infinispan.remoting.transport.Address; @@ -37,15 +38,18 @@ public abstract class AbstractEnlistmentAdapter { private final boolean isPessimisticLocking; private final boolean isTotalOrder; protected final TransactionCoordinator txCoordinator; + protected final PartitionHandlingManager partitionHandlingManager; public AbstractEnlistmentAdapter(CacheTransaction cacheTransaction, - CommandsFactory commandsFactory, RpcManager rpcManager, - TransactionTable txTable, ClusteringDependentLogic clusteringLogic, - Configuration configuration, TransactionCoordinator txCoordinator) { + CommandsFactory commandsFactory, RpcManager rpcManager, + TransactionTable txTable, ClusteringDependentLogic clusteringLogic, + Configuration configuration, TransactionCoordinator txCoordinator, + PartitionHandlingManager partitionHandlingManager) { this.commandsFactory = commandsFactory; this.rpcManager = rpcManager; this.txTable = txTable; this.clusteringLogic = clusteringLogic; + this.partitionHandlingManager = partitionHandlingManager; this.isSecondPhaseAsync = Configurations.isSecondPhaseAsync(configuration); this.isPessimisticLocking = configuration.transaction().lockingMode() == LockingMode.PESSIMISTIC; this.isTotalOrder = configuration.transaction().transactionProtocol().isTotalOrder(); @@ -54,12 +58,14 @@ public AbstractEnlistmentAdapter(CacheTransaction cacheTransaction, } public AbstractEnlistmentAdapter(CommandsFactory commandsFactory, - RpcManager rpcManager, TransactionTable txTable, - ClusteringDependentLogic clusteringLogic, Configuration configuration, TransactionCoordinator txCoordinator) { + RpcManager rpcManager, TransactionTable txTable, + ClusteringDependentLogic clusteringLogic, Configuration configuration, TransactionCoordinator txCoordinator, + PartitionHandlingManager partitionHandlingManager) { this.commandsFactory = commandsFactory; this.rpcManager = rpcManager; this.txTable = txTable; this.clusteringLogic = clusteringLogic; + this.partitionHandlingManager = partitionHandlingManager; this.isSecondPhaseAsync = Configurations.isSecondPhaseAsync(configuration); this.isPessimisticLocking = configuration.transaction().lockingMode() == LockingMode.PESSIMISTIC; this.isTotalOrder = configuration.transaction().transactionProtocol().isTotalOrder(); @@ -79,7 +85,7 @@ protected final void releaseLocksForCompletedTransaction(LocalTransaction localT } private void removeTransactionInfoRemotely(LocalTransaction localTransaction, GlobalTransaction gtx) { - if (mayHaveRemoteLocks(localTransaction) && !isSecondPhaseAsync) { + if (mayHaveRemoteLocks(localTransaction) && !isSecondPhaseAsync && !partitionHandlingManager.isTransactionPartiallyCommitted(gtx)) { final TxCompletionNotificationCommand command = commandsFactory.buildTxCompletionNotificationCommand(null, gtx); final Collection
owners = clusteringLogic.getOwners(filterDeltaCompositeKeys(localTransaction.getAffectedKeys())); Collection
commitNodes = localTransaction.getCommitNodes(owners, rpcManager.getTopologyId(), rpcManager.getMembers()); diff --git a/core/src/main/java/org/infinispan/transaction/impl/TransactionTable.java b/core/src/main/java/org/infinispan/transaction/impl/TransactionTable.java index 6f6c16fd972..2c3ad84ec44 100644 --- a/core/src/main/java/org/infinispan/transaction/impl/TransactionTable.java +++ b/core/src/main/java/org/infinispan/transaction/impl/TransactionTable.java @@ -27,6 +27,7 @@ import org.infinispan.notifications.cachemanagerlistener.CacheManagerNotifier; import org.infinispan.notifications.cachemanagerlistener.annotation.ViewChanged; import org.infinispan.notifications.cachemanagerlistener.event.ViewChangedEvent; +import org.infinispan.partitionhandling.impl.PartitionHandlingManager; import org.infinispan.remoting.rpc.RpcManager; import org.infinispan.remoting.transport.Address; import org.infinispan.transaction.LockingMode; @@ -88,19 +89,21 @@ public enum CompletedTransactionStatus { protected RpcManager rpcManager; protected CommandsFactory commandsFactory; protected ClusteringDependentLogic clusteringLogic; - protected boolean clustered = false; - private ConcurrentMap localTransactions; - private ConcurrentMap globalToLocalTransactions; - private ConcurrentMap remoteTransactions; private InterceptorChain invoker; private CacheNotifier notifier; private TransactionSynchronizationRegistry transactionSynchronizationRegistry; - private Lock minTopologyRecalculationLock; private CompletedTransactionsInfo completedTransactionsInfo; private ScheduledExecutorService executorService; private String cacheName; private TimeService timeService; private CacheManagerNotifier cacheManagerNotifier; + protected PartitionHandlingManager partitionHandlingManager; + + private ConcurrentMap localTransactions; + private ConcurrentMap globalToLocalTransactions; + private ConcurrentMap remoteTransactions; + private Lock minTopologyRecalculationLock; + protected boolean clustered = false; @Inject public void initialize(RpcManager rpcManager, Configuration configuration, @@ -108,7 +111,7 @@ public void initialize(RpcManager rpcManager, Configuration configuration, TransactionFactory gtf, TransactionCoordinator txCoordinator, TransactionSynchronizationRegistry transactionSynchronizationRegistry, CommandsFactory commandsFactory, ClusteringDependentLogic clusteringDependentLogic, Cache cache, - TimeService timeService, CacheManagerNotifier cacheManagerNotifier) { + TimeService timeService, CacheManagerNotifier cacheManagerNotifier, PartitionHandlingManager partitionHandlingManager) { this.rpcManager = rpcManager; this.configuration = configuration; this.icf = icf; @@ -122,6 +125,7 @@ public void initialize(RpcManager rpcManager, Configuration configuration, this.cacheManagerNotifier = cacheManagerNotifier; this.cacheName = cache.getName(); this.timeService = timeService; + this.partitionHandlingManager = partitionHandlingManager; this.clustered = configuration.clustering().cacheMode().isClustered(); } @@ -231,7 +235,7 @@ public void enlist(Transaction transaction, LocalTransaction localTransaction) { if (!localTransaction.isEnlisted()) { SynchronizationAdapter sync = new SynchronizationAdapter( localTransaction, txCoordinator, commandsFactory, rpcManager, - this, clusteringLogic, configuration); + this, clusteringLogic, configuration, partitionHandlingManager); if (transactionSynchronizationRegistry != null) { try { transactionSynchronizationRegistry.registerInterposedSynchronization(sync); @@ -281,10 +285,9 @@ public void cleanupLeaverTransactions(List
members) { if (trace) log.tracef("Checking for transactions originated on leavers. Current cache members are %s, remote transactions: %d", members, remoteTransactions.size()); HashSet
membersSet = new HashSet<>(members); - List toKill = new ArrayList(); + List toKill = new ArrayList<>(); for (Map.Entry e : remoteTransactions.entrySet()) { GlobalTransaction gt = e.getKey(); - RemoteTransaction remoteTx = e.getValue(); if (trace) log.tracef("Checking transaction %s", gt); if (!membersSet.contains(gt.getAddress())) { toKill.add(gt); @@ -295,22 +298,25 @@ public void cleanupLeaverTransactions(List
members) { if (trace) log.tracef("No remote transactions pertain to originator(s) who have left the cluster."); } else { log.debugf("The originating node left the cluster for %d remote transactions", toKill.size()); - } + for (GlobalTransaction gtx : toKill) { + if (partitionHandlingManager.canRollbackTransactionAfterOriginatorLeave(gtx)) { + log.debugf("Rolling back transaction %s because originator %s left the cluster", gtx, gtx.getAddress()); + killTransaction(gtx); + } else { + log.debugf("Keeping transaction %s after the originator %s left the cluster.", gtx, gtx.getAddress()); + } + } - for (GlobalTransaction gtx : toKill) { - log.debugf("Rolling back transaction %s because originator %s left the cluster", gtx, gtx.getAddress()); - killTransaction(gtx); + if (trace) log.tracef("Completed cleaning transactions originating on leavers. Remote transactions remaining: %d", + remoteTransactions.size()); } - - if (trace) log.tracef("Completed cleaning transactions originating on leavers. Remote transactions remaining: %d", - remoteTransactions.size()); } public void cleanupTimedOutTransactions() { if (trace) log.tracef("About to cleanup remote transactions older than %d ms", configuration.transaction().completedTxTimeout()); long beginning = timeService.time(); long cutoffCreationTime = beginning - TimeUnit.MILLISECONDS.toNanos(configuration.transaction().completedTxTimeout()); - List toKill = new ArrayList(); + List toKill = new ArrayList<>(); // Check remote transactions. for(Map.Entry e : remoteTransactions.entrySet()) { @@ -615,10 +621,8 @@ public void markTransactionCompleted(GlobalTransaction gtx, boolean successful) * @see #markTransactionCompleted(org.infinispan.transaction.xa.GlobalTransaction, boolean) */ public boolean isTransactionCompleted(GlobalTransaction gtx) { - if (completedTransactionsInfo == null) - return false; + return completedTransactionsInfo != null && completedTransactionsInfo.isTransactionCompleted(gtx); - return completedTransactionsInfo.isTransactionCompleted(gtx); } /** diff --git a/core/src/main/java/org/infinispan/transaction/synchronization/SynchronizationAdapter.java b/core/src/main/java/org/infinispan/transaction/synchronization/SynchronizationAdapter.java index c3bf0e2523c..014b8e260b8 100644 --- a/core/src/main/java/org/infinispan/transaction/synchronization/SynchronizationAdapter.java +++ b/core/src/main/java/org/infinispan/transaction/synchronization/SynchronizationAdapter.java @@ -4,6 +4,7 @@ import org.infinispan.commons.CacheException; import org.infinispan.configuration.cache.Configuration; import org.infinispan.interceptors.locking.ClusteringDependentLogic; +import org.infinispan.partitionhandling.impl.PartitionHandlingManager; import org.infinispan.remoting.rpc.RpcManager; import org.infinispan.transaction.impl.AbstractEnlistmentAdapter; import org.infinispan.transaction.impl.LocalTransaction; @@ -32,8 +33,8 @@ public class SynchronizationAdapter extends AbstractEnlistmentAdapter implements public SynchronizationAdapter(LocalTransaction localTransaction, TransactionCoordinator txCoordinator, CommandsFactory commandsFactory, RpcManager rpcManager, TransactionTable transactionTable, ClusteringDependentLogic clusteringLogic, - Configuration configuration) { - super(localTransaction, commandsFactory, rpcManager, transactionTable, clusteringLogic, configuration, txCoordinator); + Configuration configuration, PartitionHandlingManager partitionHandlingManager) { + super(localTransaction, commandsFactory, rpcManager, transactionTable, clusteringLogic, configuration, txCoordinator, partitionHandlingManager); this.localTransaction = localTransaction; } diff --git a/core/src/main/java/org/infinispan/transaction/tm/DummyTransaction.java b/core/src/main/java/org/infinispan/transaction/tm/DummyTransaction.java index 0d26f3a0aac..7fd6bc4be7e 100644 --- a/core/src/main/java/org/infinispan/transaction/tm/DummyTransaction.java +++ b/core/src/main/java/org/infinispan/transaction/tm/DummyTransaction.java @@ -102,9 +102,7 @@ public void commit() throws RollbackException, HeuristicMixedException, Heuristi checkDone("Cannot commit transaction."); runPrepare(); runCommit(false); - if (firstRollbackException != null) { - throw firstRollbackException; - } + throwRollbackExceptionIfAny(); } /** @@ -341,6 +339,12 @@ public final boolean equals(Object obj) { return this == obj; } + public final void throwRollbackExceptionIfAny() throws RollbackException { + if (firstRollbackException != null) { + throw firstRollbackException; + } + } + private void markRollbackOnly(RollbackException e) { if (status == Status.STATUS_MARKED_ROLLBACK) { return; diff --git a/core/src/main/java/org/infinispan/transaction/xa/TransactionXaAdapter.java b/core/src/main/java/org/infinispan/transaction/xa/TransactionXaAdapter.java index b83ba6f6ad1..eaa7bcd4eaf 100644 --- a/core/src/main/java/org/infinispan/transaction/xa/TransactionXaAdapter.java +++ b/core/src/main/java/org/infinispan/transaction/xa/TransactionXaAdapter.java @@ -3,6 +3,7 @@ import org.infinispan.commands.CommandsFactory; import org.infinispan.configuration.cache.Configuration; import org.infinispan.interceptors.locking.ClusteringDependentLogic; +import org.infinispan.partitionhandling.impl.PartitionHandlingManager; import org.infinispan.remoting.rpc.RpcManager; import org.infinispan.transaction.impl.AbstractEnlistmentAdapter; import org.infinispan.transaction.impl.TransactionCoordinator; @@ -55,8 +56,8 @@ public TransactionXaAdapter(LocalXaTransaction localTransaction, TransactionTabl RecoveryManager rm, TransactionCoordinator txCoordinator, CommandsFactory commandsFactory, RpcManager rpcManager, ClusteringDependentLogic clusteringDependentLogic, - Configuration configuration, String cacheName) { - super(localTransaction, commandsFactory, rpcManager, txTable, clusteringDependentLogic, configuration, txCoordinator); + Configuration configuration, String cacheName, PartitionHandlingManager partitionHandlingManager) { + super(localTransaction, commandsFactory, rpcManager, txTable, clusteringDependentLogic, configuration, txCoordinator, partitionHandlingManager); this.localTransaction = localTransaction; this.txTable = (XaTransactionTable) txTable; this.recoveryManager = rm; @@ -66,12 +67,13 @@ public TransactionXaAdapter(LocalXaTransaction localTransaction, TransactionTabl this.onePhaseTotalOrder = configuration.transaction().transactionProtocol().isTotalOrder() && !(configuration.clustering().cacheMode().isDistributed() && configuration.locking().writeSkewCheck()); } + public TransactionXaAdapter(TransactionTable txTable, RecoveryManager rm, TransactionCoordinator txCoordinator, CommandsFactory commandsFactory, RpcManager rpcManager, ClusteringDependentLogic clusteringDependentLogic, - Configuration configuration, String cacheName) { - super(commandsFactory, rpcManager, txTable, clusteringDependentLogic, configuration, txCoordinator); + Configuration configuration, String cacheName, PartitionHandlingManager partitionHandlingManager) { + super(commandsFactory, rpcManager, txTable, clusteringDependentLogic, configuration, txCoordinator, partitionHandlingManager); localTransaction = null; this.txTable = (XaTransactionTable) txTable; this.recoveryManager = rm; @@ -145,7 +147,7 @@ public void forget(Xid externalXid) throws XAException { if (trace) log.tracef("forget called for xid %s", xid); try { if (recoveryEnabled) { - recoveryManager.removeRecoveryInformationFromCluster(null, xid, true, null); + recoveryManager.removeRecoveryInformation(null, xid, true, null, false); } else { if (trace) log.trace("Recovery not enabled"); } @@ -221,7 +223,8 @@ public String toString() { private void forgetSuccessfullyCompletedTransaction(RecoveryManager recoveryManager, Xid xid, LocalXaTransaction localTransaction, boolean committedInOnePhase) { final GlobalTransaction gtx = localTransaction.getGlobalTransaction(); if (recoveryEnabled) { - recoveryManager.removeRecoveryInformationFromCluster(localTransaction.getRemoteLocksAcquired(), xid, false, gtx); + recoveryManager.removeRecoveryInformation(localTransaction.getRemoteLocksAcquired(), xid, false, gtx, + partitionHandlingManager.isTransactionPartiallyCommitted(gtx)); txTable.removeLocalTransaction(localTransaction); } else { releaseLocksForCompletedTransaction(localTransaction, committedInOnePhase); diff --git a/core/src/main/java/org/infinispan/transaction/xa/XaTransactionTable.java b/core/src/main/java/org/infinispan/transaction/xa/XaTransactionTable.java index 8c52bd7a48f..67de5be463c 100644 --- a/core/src/main/java/org/infinispan/transaction/xa/XaTransactionTable.java +++ b/core/src/main/java/org/infinispan/transaction/xa/XaTransactionTable.java @@ -76,7 +76,7 @@ public void enlist(Transaction transaction, LocalTransaction ltx) { transaction.enlistResource(new TransactionXaAdapter( localTransaction, this, recoveryManager, txCoordinator, commandsFactory, rpcManager, - clusteringLogic, configuration, cacheName)); + clusteringLogic, configuration, cacheName, partitionHandlingManager)); } catch (Exception e) { Xid xid = localTransaction.getXid(); if (xid != null && !localTransaction.getLookedUpEntries().isEmpty()) { diff --git a/core/src/main/java/org/infinispan/transaction/xa/recovery/RecoveryAdminOperations.java b/core/src/main/java/org/infinispan/transaction/xa/recovery/RecoveryAdminOperations.java index 0bc558dec05..b31f62c0865 100644 --- a/core/src/main/java/org/infinispan/transaction/xa/recovery/RecoveryAdminOperations.java +++ b/core/src/main/java/org/infinispan/transaction/xa/recovery/RecoveryAdminOperations.java @@ -92,7 +92,7 @@ public String forget( @Parameter(name = "formatId", description = "The formatId of the transaction") int formatId, @Parameter(name = "globalTxId", description = "The globalTxId of the transaction") byte[] globalTxId, @Parameter(name = "branchQualifier", description = "The branchQualifier of the transaction") byte[] branchQualifier) { - recoveryManager.removeRecoveryInformationFromCluster(null, new SerializableXid(branchQualifier, globalTxId, formatId), true, null); + recoveryManager.removeRecoveryInformation(null, new SerializableXid(branchQualifier, globalTxId, formatId), true, null, false); return "Recovery info removed."; } diff --git a/core/src/main/java/org/infinispan/transaction/xa/recovery/RecoveryManager.java b/core/src/main/java/org/infinispan/transaction/xa/recovery/RecoveryManager.java index d96a9d3ee13..1ed66f52258 100644 --- a/core/src/main/java/org/infinispan/transaction/xa/recovery/RecoveryManager.java +++ b/core/src/main/java/org/infinispan/transaction/xa/recovery/RecoveryManager.java @@ -20,20 +20,24 @@ public interface RecoveryManager { /** - * Returns the list of transactions in prepared state from both local and remote cluster nodes. - * Implementation can take advantage of several optimisations: - *
-    * - in order to get all tx from the cluster a broadcast is performed. This can be performed only once (assuming the call
-    *   is successful), the first time this method is called. After that a local, cached list of tx prepared on this node is returned.
-    * - during the broadcast just return the list of prepared transactions that are not originated on other active nodes of the
-    * cluster.
-    * 
+ * Returns the list of transactions in prepared state from both local and remote cluster nodes. Implementation can + * take advantage of several optimisations: + * + *
    + *
  • in order to get all tx from the cluster a broadcast is performed. This can be performed only once + * (assuming the call is successful), the first time this method is called. After that a local, cached list of tx + * prepared on this node is returned.
  • + *
  • during the broadcast just return the list of prepared transactions that are not originated on other active + * nodes of the cluster.
  • + *
*/ RecoveryIterator getPreparedTransactionsFromCluster(); /** - * Returns a {@link Set} containing all the in-doubt transactions from the cluster, including the local node. This does - * not include transactions that are prepared successfully and for which the originator is still in the cluster. + * Returns a {@link Set} containing all the in-doubt transactions from the cluster, including the local node. This + * does not include transactions that are prepared successfully and for which the originator is still in the + * cluster. + * * @see InDoubtTxInfo */ Set getInDoubtTransactionInfoFromCluster(); @@ -45,18 +49,23 @@ public interface RecoveryManager { /** - * Removes from the specified nodes (or all nodes if the value of 'where' is null) the recovery information associated with - * these Xids. - * @param where on which nodes should this be executed. - * @param xid the list of xids to be removed. - * @param sync execute sync or async (false) - * @param gtx + * Removes from the specified nodes (or all nodes if the value of 'where' is null) the recovery information + * associated with these Xids. + * + * @param skipTxCompletionCommand {@code true} if it must skip the {@link org.infinispan.commands.remote.recovery.TxCompletionNotificationCommand}. + * Used when a partition happens. + * @param where on which nodes should this be executed. + * @param xid the list of xids to be removed. + * @param sync execute sync or async (false) + * @param gtx the global transaction + * @param fromCluster */ - void removeRecoveryInformationFromCluster(Collection
where, Xid xid, boolean sync, GlobalTransaction gtx); + void removeRecoveryInformation(Collection
where, Xid xid, boolean sync, GlobalTransaction gtx, boolean fromCluster); /** - * Same as {@link #removeRecoveryInformationFromCluster(java.util.Collection} but the transaction - * is identified by its internal id, and not by its xid. + * Same as {@link #removeRecoveryInformation(java.util.Collection, javax.transaction.xa.Xid, boolean, + * org.infinispan.transaction.xa.GlobalTransaction, boolean)} but the transaction is identified by its internal id, + * and not by its xid. */ void removeRecoveryInformationFromCluster(Collection
where, long internalId, boolean sync); @@ -81,7 +90,8 @@ public interface RecoveryManager { /** * Replays the given transaction by re-running the prepare and commit. This call expects the transaction to exist on * this node either as a local or remote transaction. - * @param xid tx to commit or rollback + * + * @param xid tx to commit or rollback * @param commit if true tx is committed, if false it is rolled back */ String forceTransactionCompletion(Xid xid, boolean commit); @@ -105,14 +115,14 @@ public interface RecoveryManager { /** * Remove recovery information stored on this node (doesn't involve rpc). * - * @param xid@see #removeRecoveryInformation(java.util.Collection, javax.transaction.xa.Xid, boolean) + * @param xid@see #removeRecoveryInformation(java.util.Collection, javax.transaction.xa.Xid, boolean) */ RecoveryAwareTransaction removeRecoveryInformation(Xid xid); /** - * Stateful structure allowing prepared-tx retrieval in a batch-oriented manner, - * as required by {@link javax.transaction.xa.XAResource#recover(int)}. - */ + * Stateful structure allowing prepared-tx retrieval in a batch-oriented manner, as required by {@link + * javax.transaction.xa.XAResource#recover(int)}. + */ interface RecoveryIterator extends Iterator { Xid[] NOTHING = new Xid[]{}; @@ -140,8 +150,9 @@ interface InDoubtTxInfo { Long getInternalId(); /** - * The value represent transaction's state as described by the {@link Status} field. Multiple values are returned - * as it is possible for an in-doubt transaction to be at the same time e.g. prepared on one node and committed on the other. + * The value represent transaction's state as described by the {@code status} field. Multiple values are returned + * as it is possible for an in-doubt transaction to be at the same time e.g. prepared on one node and committed on + * the other. */ Set getStatus(); diff --git a/core/src/main/java/org/infinispan/transaction/xa/recovery/RecoveryManagerImpl.java b/core/src/main/java/org/infinispan/transaction/xa/recovery/RecoveryManagerImpl.java index 289d960d078..5a973014fdf 100644 --- a/core/src/main/java/org/infinispan/transaction/xa/recovery/RecoveryManagerImpl.java +++ b/core/src/main/java/org/infinispan/transaction/xa/recovery/RecoveryManagerImpl.java @@ -123,10 +123,11 @@ public RecoveryIterator getPreparedTransactionsFromCluster() { } @Override - public void removeRecoveryInformationFromCluster(Collection
lockOwners, Xid xid, boolean sync, GlobalTransaction gtx) { + public void removeRecoveryInformation(Collection
lockOwners, Xid xid, boolean sync, GlobalTransaction gtx, + boolean fromCluster) { log.tracef("Forgetting tx information for %s", gtx); //todo make sure this gets broad casted or at least flushed - if (rpcManager != null) { + if (rpcManager != null && !fromCluster) { TxCompletionNotificationCommand ftc = commandFactory.buildTxCompletionNotificationCommand(xid, gtx); rpcManager.invokeRemotely(lockOwners, ftc, rpcManager.getDefaultRpcOptions(sync, DeliverOrder.NONE)); } @@ -302,7 +303,7 @@ private String completeTransaction(LocalTransaction localTx, boolean commit, Xid return "Could not commit transaction " + xid + " : " + e.getMessage(); } } - removeRecoveryInformationFromCluster(null, xid, false, localTx.getGlobalTransaction()); + removeRecoveryInformation(null, xid, false, localTx.getGlobalTransaction(), false); return commit ? "Commit successful!" : "Rollback successful"; } diff --git a/core/src/test/java/org/infinispan/api/ForceWriteLockTest.java b/core/src/test/java/org/infinispan/api/ForceWriteLockTest.java index 7c13f41da06..94568b5a970 100644 --- a/core/src/test/java/org/infinispan/api/ForceWriteLockTest.java +++ b/core/src/test/java/org/infinispan/api/ForceWriteLockTest.java @@ -2,17 +2,13 @@ import org.infinispan.AdvancedCache; import org.infinispan.configuration.cache.ConfigurationBuilder; -import org.infinispan.container.entries.CacheEntry; -import org.infinispan.container.entries.ReadCommittedEntry; import org.infinispan.context.Flag; -import org.infinispan.context.InvocationContext; import org.infinispan.manager.EmbeddedCacheManager; import org.infinispan.test.SingleCacheManagerTest; import org.infinispan.test.TestingUtil; import org.infinispan.test.fwk.TestCacheManagerFactory; import org.infinispan.transaction.LockingMode; import org.infinispan.transaction.impl.LocalTransaction; -import org.infinispan.transaction.impl.TransactionCoordinator; import org.infinispan.transaction.impl.TransactionTable; import org.testng.annotations.Test; @@ -26,21 +22,21 @@ @Test(groups = "functional", testName = "api.ForceWriteLockTest") public class ForceWriteLockTest extends SingleCacheManagerTest { private TransactionManager tm; - private AdvancedCache advancedCache; + private AdvancedCache advancedCache; @Override protected EmbeddedCacheManager createCacheManager() throws Exception { ConfigurationBuilder cacheConfiguration = TestCacheManagerFactory.getDefaultCacheConfiguration(true); cacheConfiguration.transaction().lockingMode(LockingMode.PESSIMISTIC); EmbeddedCacheManager cacheManager = TestCacheManagerFactory.createCacheManager(cacheConfiguration); - advancedCache = cacheManager.getCache().getAdvancedCache(); + advancedCache = cacheManager.getCache().getAdvancedCache(); tm = TestingUtil.getTransactionManager(advancedCache); return cacheManager; } public void testWriteLockIsAcquired() throws Exception { advancedCache.put("k","v"); - assertNotLocked(advancedCache,"k"); + assertEventuallyNotLocked(advancedCache, "k"); tm.begin(); advancedCache.withFlags(Flag.FORCE_WRITE_LOCK).get("k"); @@ -50,6 +46,6 @@ public void testWriteLockIsAcquired() throws Exception { assertLocked(advancedCache,"k"); tm.commit(); - assertNotLocked(advancedCache,"k"); + assertEventuallyNotLocked(advancedCache, "k"); } } diff --git a/core/src/test/java/org/infinispan/lock/ExplicitUnlockTest.java b/core/src/test/java/org/infinispan/lock/ExplicitUnlockTest.java index c247481d049..78a19a88b8c 100644 --- a/core/src/test/java/org/infinispan/lock/ExplicitUnlockTest.java +++ b/core/src/test/java/org/infinispan/lock/ExplicitUnlockTest.java @@ -67,7 +67,7 @@ private void doTestLock(boolean withUnlock, int nThreads, long stepDelayMsec) th cache.put("" + key, "value"); } - List> results = new ArrayList>(nThreads); + List> results = new ArrayList<>(nThreads); for (int i = 1; i <= nThreads; i++) { results.add(fork(new Worker(i, cache, withUnlock, stepDelayMsec))); @@ -80,7 +80,7 @@ private void doTestLock(boolean withUnlock, int nThreads, long stepDelayMsec) th assertTrue("All worker should complete without exceptions", success); assertNoTransactions(); for (int i = 0; i < NUMBER_OF_KEYS; ++i) { - assertNotLocked(cache, valueOf(i)); + assertEventuallyNotLocked(cache, valueOf(i)); } } diff --git a/core/src/test/java/org/infinispan/lock/OptimisticTxFailureAfterLockingTest.java b/core/src/test/java/org/infinispan/lock/OptimisticTxFailureAfterLockingTest.java index aae9d77b849..f02a614deb8 100644 --- a/core/src/test/java/org/infinispan/lock/OptimisticTxFailureAfterLockingTest.java +++ b/core/src/test/java/org/infinispan/lock/OptimisticTxFailureAfterLockingTest.java @@ -77,6 +77,6 @@ private void doTest(int primaryOwnerIndex, int execIndex) throws Exception { } assertNoTransactions(); - assertNotLocked(cache(primaryOwnerIndex), key); + assertEventuallyNotLocked(cache(primaryOwnerIndex), key); } } diff --git a/core/src/test/java/org/infinispan/lock/PessimistTxFailureAfterLockingTest.java b/core/src/test/java/org/infinispan/lock/PessimistTxFailureAfterLockingTest.java index 7fc334b6db0..2dbd10e528e 100644 --- a/core/src/test/java/org/infinispan/lock/PessimistTxFailureAfterLockingTest.java +++ b/core/src/test/java/org/infinispan/lock/PessimistTxFailureAfterLockingTest.java @@ -66,7 +66,7 @@ private void doTest(boolean explicitLocking) throws Exception { assertTrue("Expected an exception", failed); assertNoTransactions(); - assertNotLocked(cache(1), key); + assertEventuallyNotLocked(cache(1), key); } @Override diff --git a/core/src/test/java/org/infinispan/lock/StaleEagerLocksOnPrepareFailureTest.java b/core/src/test/java/org/infinispan/lock/StaleEagerLocksOnPrepareFailureTest.java index 92428a9593b..882af9bd871 100644 --- a/core/src/test/java/org/infinispan/lock/StaleEagerLocksOnPrepareFailureTest.java +++ b/core/src/test/java/org/infinispan/lock/StaleEagerLocksOnPrepareFailureTest.java @@ -85,10 +85,10 @@ private void doTest(boolean mods) throws Exception { // expected } - assertNotLocked(c1, k1); - assertNotLocked(c2, k1); - assertNotLocked(c1, k2); - assertNotLocked(c2, k2); + assertEventuallyNotLocked(c1, k1); + assertEventuallyNotLocked(c2, k1); + assertEventuallyNotLocked(c1, k2); + assertEventuallyNotLocked(c2, k2); } } diff --git a/core/src/test/java/org/infinispan/lock/StaleLocksOnPrepareFailureTest.java b/core/src/test/java/org/infinispan/lock/StaleLocksOnPrepareFailureTest.java index 62f05e28cde..913a7e745ca 100644 --- a/core/src/test/java/org/infinispan/lock/StaleLocksOnPrepareFailureTest.java +++ b/core/src/test/java/org/infinispan/lock/StaleLocksOnPrepareFailureTest.java @@ -58,7 +58,7 @@ private void doTest(boolean mods) throws Exception { } for (int i = 0; i < NUM_CACHES; i++) { - assertNotLocked(cache(i), k1); + assertEventuallyNotLocked(cache(i), k1); } } } diff --git a/core/src/test/java/org/infinispan/lock/StaleLocksTransactionTest.java b/core/src/test/java/org/infinispan/lock/StaleLocksTransactionTest.java index 5570469492e..673773f34b6 100644 --- a/core/src/test/java/org/infinispan/lock/StaleLocksTransactionTest.java +++ b/core/src/test/java/org/infinispan/lock/StaleLocksTransactionTest.java @@ -55,7 +55,7 @@ private void doTest(boolean mods, boolean commit) throws Exception { else tm(c1).rollback(); - assertNotLocked(c1, "k"); - assertNotLocked(c2, "k"); + assertEventuallyNotLocked(c1, "k"); + assertEventuallyNotLocked(c2, "k"); } } diff --git a/core/src/test/java/org/infinispan/lock/singlelock/AbstractInitiatorCrashTest.java b/core/src/test/java/org/infinispan/lock/singlelock/AbstractInitiatorCrashTest.java index 8e3512f10db..ad194553aee 100644 --- a/core/src/test/java/org/infinispan/lock/singlelock/AbstractInitiatorCrashTest.java +++ b/core/src/test/java/org/infinispan/lock/singlelock/AbstractInitiatorCrashTest.java @@ -31,8 +31,8 @@ public void testInitiatorCrashesBeforeReleasingLock() throws Exception { assert checkTxCount(1, 0, 0); assert checkTxCount(2, 0, 1); - assertNotLocked(cache(0), k); - assertNotLocked(cache(1), k); + assertEventuallyNotLocked(cache(0), k); + assertEventuallyNotLocked(cache(1), k); assertLocked(cache(2), k); killMember(1); @@ -56,8 +56,8 @@ public void testInitiatorNodeCrashesBeforeCommit() throws Exception { transaction.runPrepare(); tm(1).suspend(); - assertNotLocked(cache(0), k); - assertNotLocked(cache(1), k); + assertEventuallyNotLocked(cache(0), k); + assertEventuallyNotLocked(cache(1), k); assertLocked(cache(2), k); checkTxCount(0, 0, 1); diff --git a/core/src/test/java/org/infinispan/lock/singlelock/MainOwnerChangesPessimisticLockTest.java b/core/src/test/java/org/infinispan/lock/singlelock/MainOwnerChangesPessimisticLockTest.java index 2a8528a9248..306d0be3f57 100644 --- a/core/src/test/java/org/infinispan/lock/singlelock/MainOwnerChangesPessimisticLockTest.java +++ b/core/src/test/java/org/infinispan/lock/singlelock/MainOwnerChangesPessimisticLockTest.java @@ -62,7 +62,7 @@ public void testRemoteLockMigrationTxRollback() throws Exception { } private void testLockMigration(int nodeThatPuts, boolean commit) throws Exception { - Map key2Tx = new HashMap(); + Map key2Tx = new HashMap<>(); for (int i = 0; i < NUM_KEYS; i++) { Object key = getKeyForCache(0); if (key2Tx.containsKey(key)) continue; @@ -123,9 +123,9 @@ private void testLockMigration(int nodeThatPuts, boolean commit) throws Exceptio } // there should not be any locks - assertNotLocked(cache(0), migratedKey); - assertNotLocked(cache(1), migratedKey); - assertNotLocked(cache(2), migratedKey); + assertEventuallyNotLocked(cache(0), migratedKey); + assertEventuallyNotLocked(cache(1), migratedKey); + assertEventuallyNotLocked(cache(2), migratedKey); // if a new TX tries to write to the migrated key this should not fail, the key should not be locked tm(nodeThatPuts).begin(); diff --git a/core/src/test/java/org/infinispan/lock/singlelock/OriginatorBecomesOwnerLockTest.java b/core/src/test/java/org/infinispan/lock/singlelock/OriginatorBecomesOwnerLockTest.java index 35f894301f4..8a1de421572 100644 --- a/core/src/test/java/org/infinispan/lock/singlelock/OriginatorBecomesOwnerLockTest.java +++ b/core/src/test/java/org/infinispan/lock/singlelock/OriginatorBecomesOwnerLockTest.java @@ -209,7 +209,7 @@ public DummyTransaction call() throws Exception { private void assertNoLocksOrTxs(Object key, Cache cache) { - assertNotLocked(originatorCache, key); + assertEventuallyNotLocked(originatorCache, key); final TransactionTable transactionTable = TestingUtil.extractComponent(cache, TransactionTable.class); diff --git a/core/src/test/java/org/infinispan/lock/singlelock/pessimistic/InitiatorCrashPessimisticTest.java b/core/src/test/java/org/infinispan/lock/singlelock/pessimistic/InitiatorCrashPessimisticTest.java index b6568284550..a4fb3e11beb 100644 --- a/core/src/test/java/org/infinispan/lock/singlelock/pessimistic/InitiatorCrashPessimisticTest.java +++ b/core/src/test/java/org/infinispan/lock/singlelock/pessimistic/InitiatorCrashPessimisticTest.java @@ -31,15 +31,15 @@ public void testInitiatorNodeCrashesBeforePrepare2() throws Exception { cache(1).put(k2, "v2"); assertLocked(cache(0), k0); - assertNotLocked(cache(1), k0); - assertNotLocked(cache(2), k0); + assertEventuallyNotLocked(cache(1), k0); + assertEventuallyNotLocked(cache(2), k0); - assertNotLocked(cache(0), k1); + assertEventuallyNotLocked(cache(0), k1); assertLocked(cache(1), k1); - assertNotLocked(cache(2), k1); + assertEventuallyNotLocked(cache(2), k1); - assertNotLocked(cache(0), k2); - assertNotLocked(cache(1), k2); + assertEventuallyNotLocked(cache(0), k2); + assertEventuallyNotLocked(cache(1), k2); assertLocked(cache(2), k2); assert checkTxCount(0, 0, 1); diff --git a/core/src/test/java/org/infinispan/lock/singlelock/replicated/optimistic/InitiatorCrashOptimisticReplTest.java b/core/src/test/java/org/infinispan/lock/singlelock/replicated/optimistic/InitiatorCrashOptimisticReplTest.java index 954948083e5..d92ac426919 100644 --- a/core/src/test/java/org/infinispan/lock/singlelock/replicated/optimistic/InitiatorCrashOptimisticReplTest.java +++ b/core/src/test/java/org/infinispan/lock/singlelock/replicated/optimistic/InitiatorCrashOptimisticReplTest.java @@ -34,8 +34,8 @@ public void testInitiatorNodeCrashesBeforeCommit() throws Exception { txControlInterceptor.commitReceived.await(); assertLocked(cache(0), "k"); - assertNotLocked(cache(1), "k"); - assertNotLocked(cache(2), "k"); + assertEventuallyNotLocked(cache(1), "k"); + assertEventuallyNotLocked(cache(2), "k"); checkTxCount(0, 0, 1); checkTxCount(1, 1, 0); @@ -65,8 +65,8 @@ public void testInitiatorCrashesBeforeReleasingLock() throws Exception { assert checkTxCount(2, 0, 1); assertLocked(cache(0), "k"); - assertNotLocked(cache(1), "k"); - assertNotLocked(cache(2), "k"); + assertEventuallyNotLocked(cache(1), "k"); + assertEventuallyNotLocked(cache(2), "k"); killMember(1); diff --git a/core/src/test/java/org/infinispan/lock/singlelock/replicated/pessimistic/InitiatorCrashPessimisticReplTest.java b/core/src/test/java/org/infinispan/lock/singlelock/replicated/pessimistic/InitiatorCrashPessimisticReplTest.java index cf63a189c07..660d2f2789e 100644 --- a/core/src/test/java/org/infinispan/lock/singlelock/replicated/pessimistic/InitiatorCrashPessimisticReplTest.java +++ b/core/src/test/java/org/infinispan/lock/singlelock/replicated/pessimistic/InitiatorCrashPessimisticReplTest.java @@ -31,8 +31,8 @@ public void testInitiatorNodeCrashesBeforeCommit() throws Exception { txControlInterceptor.preparedReceived.await(); assertLocked(cache(0), key); - assertNotLocked(cache(1), key); - assertNotLocked(cache(2), key); + assertEventuallyNotLocked(cache(1), key); + assertEventuallyNotLocked(cache(2), key); checkTxCount(0, 0, 1); checkTxCount(1, 1, 0); @@ -63,8 +63,8 @@ public void testInitiatorCrashesBeforeReleasingLock() throws Exception { assert checkTxCount(2, 0, 1); assertLocked(cache(0), key); - assertNotLocked(cache(1), key); - assertNotLocked(cache(2), key); + assertEventuallyNotLocked(cache(1), key); + assertEventuallyNotLocked(cache(2), key); killMember(1); diff --git a/core/src/test/java/org/infinispan/partitionhandling/BaseOptimisticTxPartitionAndMergeTest.java b/core/src/test/java/org/infinispan/partitionhandling/BaseOptimisticTxPartitionAndMergeTest.java new file mode 100644 index 00000000000..42150dc02c7 --- /dev/null +++ b/core/src/test/java/org/infinispan/partitionhandling/BaseOptimisticTxPartitionAndMergeTest.java @@ -0,0 +1,79 @@ +package org.infinispan.partitionhandling; + +import org.infinispan.Cache; +import org.infinispan.commands.tx.TransactionBoundaryCommand; +import org.infinispan.configuration.cache.CacheMode; +import org.infinispan.configuration.cache.ConfigurationBuilder; +import org.infinispan.transaction.LockingMode; +import org.infinispan.transaction.TransactionMode; +import org.infinispan.transaction.lookup.DummyTransactionManagerLookup; +import org.infinispan.transaction.tm.DummyTransaction; +import org.infinispan.transaction.tm.DummyTransactionManager; + +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +import static org.testng.AssertJUnit.assertFalse; +import static org.testng.AssertJUnit.assertTrue; + +/** + * It tests multiple scenarios where a split can happen during a transaction. + * + * @author Pedro Ruivo + * @since 8.0 + */ +public abstract class BaseOptimisticTxPartitionAndMergeTest extends BaseTxPartitionAndMergeTest { + + protected static final String OPTIMISTIC_TX_CACHE_NAME = "opt-cache"; + + @Override + protected void createCacheManagers() throws Throwable { + super.createCacheManagers(); + ConfigurationBuilder builder = getDefaultClusteredCacheConfig(CacheMode.DIST_SYNC); + builder.clustering().partitionHandling().enabled(true); + builder.transaction().lockingMode(LockingMode.OPTIMISTIC).transactionMode(TransactionMode.TRANSACTIONAL).transactionManagerLookup(new DummyTransactionManagerLookup()); + defineConfigurationOnAllManagers(OPTIMISTIC_TX_CACHE_NAME, builder); + } + + protected abstract void checkLocksDuringPartition(SplitMode splitMode, KeyInfo keyInfo, boolean discard); + + protected abstract boolean forceRollback(); + + protected abstract Class getCommandClass(); + + protected void doTest(final SplitMode splitMode, boolean txFail, boolean discard) throws Exception { + waitForClusterToForm(OPTIMISTIC_TX_CACHE_NAME); + + final KeyInfo keyInfo = createKeys(OPTIMISTIC_TX_CACHE_NAME); + final Cache originator = cache(0, OPTIMISTIC_TX_CACHE_NAME); + final FilterCollection filterCollection = createFilters(OPTIMISTIC_TX_CACHE_NAME, discard, getCommandClass(), splitMode); + + Future put = fork(() -> { + final DummyTransactionManager transactionManager = (DummyTransactionManager) originator.getAdvancedCache().getTransactionManager(); + transactionManager.begin(); + keyInfo.putFinalValue(originator); + final DummyTransaction transaction = transactionManager.getTransaction(); + transaction.runPrepare(); + transaction.runCommit(forceRollback()); + transaction.throwRollbackExceptionIfAny(); + return null; + }); + + filterCollection.await(30, TimeUnit.SECONDS); + splitMode.split(this); + filterCollection.unblock(); + + try { + put.get(); + assertFalse(txFail); + } catch (ExecutionException e) { + assertTrue(txFail); + } + + checkLocksDuringPartition(splitMode, keyInfo, discard); + + mergeCluster(); + finalAsserts(OPTIMISTIC_TX_CACHE_NAME, keyInfo, txFail ? INITIAL_VALUE : FINAL_VALUE); + } +} diff --git a/core/src/test/java/org/infinispan/partitionhandling/BasePartitionHandlingTest.java b/core/src/test/java/org/infinispan/partitionhandling/BasePartitionHandlingTest.java index df13b6898ef..595d6c9407a 100644 --- a/core/src/test/java/org/infinispan/partitionhandling/BasePartitionHandlingTest.java +++ b/core/src/test/java/org/infinispan/partitionhandling/BasePartitionHandlingTest.java @@ -7,6 +7,8 @@ import org.infinispan.notifications.cachemanagerlistener.annotation.ViewChanged; import org.infinispan.notifications.cachemanagerlistener.event.ViewChangedEvent; import org.infinispan.partitionhandling.impl.PartitionHandlingManager; +import org.infinispan.remoting.transport.AbstractDelegatingTransport; +import org.infinispan.remoting.transport.Transport; import org.infinispan.remoting.transport.jgroups.JGroupsTransport; import org.infinispan.test.MultipleCacheManagersTest; import org.infinispan.test.TestingUtil; @@ -167,7 +169,7 @@ private View toView(ArrayList channels) { } private void discardOtherMembers() { - List
outsideMembers = new ArrayList
(); + List
outsideMembers = new ArrayList<>(); for (Address a : allMembers) { boolean inThisPartition = false; for (Channel c : channels) { @@ -270,7 +272,7 @@ public void assertKeyAvailableForRead(Object k, Object expectedValue) { } public void assertKeyAvailableForWrite(Object k, Object newValue) { - for (Cache c : cachesInThisPartition()) { + for (Cache c : cachesInThisPartition()) { c.put(k, newValue); assertEquals(c.get(k), newValue, "Cache " + c.getAdvancedCache().getRpcManager().getAddress() + " doesn't see the right value"); } @@ -282,18 +284,20 @@ protected void assertKeysNotAvailableForRead(Object... keys) { } protected void assertKeyNotAvailableForRead(Object key) { - for (Cache c : cachesInThisPartition()) { + for (Cache c : cachesInThisPartition()) { try { c.get(key); fail("Key " + key + " available in cache " + address(c)); - } catch (AvailabilityException ae) {} + } catch (AvailabilityException ae) { + //expected! + } } } - private List cachesInThisPartition() { - List caches = new ArrayList(); - for (final Cache c : caches()) { + private List> cachesInThisPartition() { + List> caches = new ArrayList<>(); + for (final Cache c : BasePartitionHandlingTest.this.caches()) { if (channels.contains(channel(c))) { caches.add(c); } @@ -302,11 +306,13 @@ private List cachesInThisPartition() { } public void assertKeyNotAvailableForWrite(Object key) { - for (Cache c : cachesInThisPartition()) { + for (Cache c : cachesInThisPartition()) { try { c.put(key, key); fail(); - } catch (AvailabilityException ae) {} + } catch (AvailabilityException ae) { + //expected! + } } } @@ -342,13 +348,11 @@ protected void splitCluster(int[]... parts) { } private Channel channel(int i) { - Cache cache = cache(i); - return channel(cache); + return channel(cache(i)); } - private Channel channel(Cache cache) { - JGroupsTransport t = (JGroupsTransport) cache.getAdvancedCache().getRpcManager().getTransport(); - return t.getChannel(); + private Channel channel(Cache cache) { + return extractJGroupsTransport(cache.getAdvancedCache().getRpcManager().getTransport()).getChannel(); } protected Partition partition(int i) { @@ -371,4 +375,13 @@ protected void assertExpectedValue(Object expectedVal, Object key) { } } + private static JGroupsTransport extractJGroupsTransport(Transport transport) { + if (transport instanceof AbstractDelegatingTransport) { + return extractJGroupsTransport(((AbstractDelegatingTransport) transport).getDelegate()); + } else if (transport instanceof JGroupsTransport) { + return (JGroupsTransport) transport; + } + throw new IllegalArgumentException("Transport is not a JGroupsTransport! It is " + transport.getClass()); + } + } diff --git a/core/src/test/java/org/infinispan/partitionhandling/BasePessimisticTxPartitionAndMergeTest.java b/core/src/test/java/org/infinispan/partitionhandling/BasePessimisticTxPartitionAndMergeTest.java new file mode 100644 index 00000000000..c83b42a013e --- /dev/null +++ b/core/src/test/java/org/infinispan/partitionhandling/BasePessimisticTxPartitionAndMergeTest.java @@ -0,0 +1,83 @@ +package org.infinispan.partitionhandling; + +import org.infinispan.Cache; +import org.infinispan.commands.tx.TransactionBoundaryCommand; +import org.infinispan.configuration.cache.CacheMode; +import org.infinispan.configuration.cache.ConfigurationBuilder; +import org.infinispan.transaction.LockingMode; +import org.infinispan.transaction.TransactionMode; +import org.infinispan.transaction.lookup.DummyTransactionManagerLookup; + +import javax.transaction.Transaction; +import javax.transaction.TransactionManager; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +import static org.testng.AssertJUnit.assertFalse; +import static org.testng.AssertJUnit.assertTrue; + +/** + * It tests multiple scenarios where a split can happen during a transaction. + * + * @author Pedro Ruivo + * @since 8.0 + */ +public abstract class BasePessimisticTxPartitionAndMergeTest extends BaseTxPartitionAndMergeTest { + + protected static final String PESSIMISTIC_TX_CACHE_NAME = "pes-cache"; + + @Override + protected void createCacheManagers() throws Throwable { + super.createCacheManagers(); + ConfigurationBuilder builder = getDefaultClusteredCacheConfig(CacheMode.DIST_SYNC); + builder.clustering().partitionHandling().enabled(true); + builder.transaction().lockingMode(LockingMode.PESSIMISTIC).transactionMode(TransactionMode.TRANSACTIONAL).transactionManagerLookup(new DummyTransactionManagerLookup()); + defineConfigurationOnAllManagers(PESSIMISTIC_TX_CACHE_NAME, builder); + } + + protected abstract void checkLocksDuringPartition(SplitMode splitMode, KeyInfo keyInfo, boolean discard); + + protected abstract boolean forceRollback(); + + protected abstract Class getCommandClass(); + + protected void doTest(final SplitMode splitMode, boolean txFail, boolean discard) throws Exception { + waitForClusterToForm(PESSIMISTIC_TX_CACHE_NAME); + + final KeyInfo keyInfo = createKeys(PESSIMISTIC_TX_CACHE_NAME); + final Cache originator = cache(0, PESSIMISTIC_TX_CACHE_NAME); + final FilterCollection filterCollection = createFilters(PESSIMISTIC_TX_CACHE_NAME, discard, getCommandClass(), splitMode); + + Future put = fork(() -> { + final TransactionManager transactionManager = originator.getAdvancedCache().getTransactionManager(); + transactionManager.begin(); + final Transaction tx = transactionManager.getTransaction(); + try { + keyInfo.putFinalValue(originator); + if (forceRollback()) { + tx.setRollbackOnly(); + } + } finally { + transactionManager.commit(); + } + return null; + }); + + filterCollection.await(30, TimeUnit.SECONDS); + splitMode.split(this); + filterCollection.unblock(); + + try { + put.get(); + assertFalse(txFail); + } catch (ExecutionException e) { + assertTrue(txFail); + } + + checkLocksDuringPartition(splitMode, keyInfo, discard); + + mergeCluster(); + finalAsserts(PESSIMISTIC_TX_CACHE_NAME, keyInfo, txFail ? INITIAL_VALUE : FINAL_VALUE); + } +} diff --git a/core/src/test/java/org/infinispan/partitionhandling/BaseTxPartitionAndMergeTest.java b/core/src/test/java/org/infinispan/partitionhandling/BaseTxPartitionAndMergeTest.java new file mode 100644 index 00000000000..75f8c5986f0 --- /dev/null +++ b/core/src/test/java/org/infinispan/partitionhandling/BaseTxPartitionAndMergeTest.java @@ -0,0 +1,334 @@ +package org.infinispan.partitionhandling; + +import org.infinispan.Cache; +import org.infinispan.commands.remote.CacheRpcCommand; +import org.infinispan.distribution.MagicKey; +import org.infinispan.partitionhandling.impl.PartitionHandlingManager; +import org.infinispan.remoting.inboundhandler.DeliverOrder; +import org.infinispan.remoting.inboundhandler.PerCacheInboundInvocationHandler; +import org.infinispan.remoting.inboundhandler.Reply; +import org.infinispan.transaction.xa.GlobalTransaction; +import org.infinispan.util.concurrent.ReclosableLatch; +import org.infinispan.util.concurrent.TimeoutException; +import org.infinispan.util.concurrent.locks.LockManager; +import org.infinispan.util.logging.Log; +import org.testng.AssertJUnit; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import static org.infinispan.test.TestingUtil.extractComponent; +import static org.infinispan.test.TestingUtil.extractLockManager; +import static org.infinispan.test.TestingUtil.wrapPerCacheInboundInvocationHandler; + +/** + * It tests multiple scenarios where a split can happen during a transaction. + * + * @author Pedro Ruivo + * @since 8.0 + */ +public abstract class BaseTxPartitionAndMergeTest extends BasePartitionHandlingTest { + + protected static final String INITIAL_VALUE = "init-value"; + protected static final String FINAL_VALUE = "final-value"; + + private static NotifierFilter notifyCommandOn(Cache cache, Class blockClass) { + NotifierFilter filter = new NotifierFilter(blockClass); + wrapAndApplyFilter(cache, filter); + return filter; + } + + private static BlockingFilter blockCommandOn(Cache cache, Class blockClass) { + BlockingFilter filter = new BlockingFilter(blockClass); + wrapAndApplyFilter(cache, filter); + return filter; + } + + private static DiscardFilter discardCommandOn(Cache cache, Class blockClass) { + DiscardFilter filter = new DiscardFilter(blockClass); + wrapAndApplyFilter(cache, filter); + return filter; + } + + private static void wrapAndApplyFilter(Cache cache, Filter filter) { + ControlledInboundHandler controlledInboundHandler = wrapPerCacheInboundInvocationHandler(cache, (wrapOn, current) -> new ControlledInboundHandler(current), true); + controlledInboundHandler.filter = filter; + } + + protected FilterCollection createFilters(String cacheName, boolean discard, Class commandClass, SplitMode splitMode) { + Collection collection = new ArrayList<>(2); + if (splitMode == SplitMode.ORIGINATOR_ISOLATED) { + if (discard) { + collection.add(discardCommandOn(cache(1, cacheName), commandClass)); + collection.add(discardCommandOn(cache(2, cacheName), commandClass)); + } else { + collection.add(blockCommandOn(cache(1, cacheName), commandClass)); + collection.add(blockCommandOn(cache(2, cacheName), commandClass)); + } + } else { + collection.add(notifyCommandOn(cache(1, cacheName), commandClass)); + if (discard) { + collection.add(discardCommandOn(cache(2, cacheName), commandClass)); + } else { + collection.add(blockCommandOn(cache(2, cacheName), commandClass)); + } + } + return new FilterCollection(collection); + } + + protected abstract Log getLog(); + + protected void mergeCluster() { + getLog().debugf("Merging cluster"); + partition(0).merge(partition(1)); + getLog().debugf("Cluster merged"); + } + + protected void finalAsserts(String cacheName, KeyInfo keyInfo, String value) { + assertNoTransactions(cacheName); + assertNoTransactionsInPartitionHandler(cacheName); + assertNoLocks(cacheName); + + assertValue(keyInfo.getKey1(), value, this.caches(cacheName)); + assertValue(keyInfo.getKey2(), value, this.caches(cacheName)); + } + + protected void assertNoLocks(String cacheName) { + eventually("Expected no locks acquired in all nodes.", () -> { + for (Cache cache : caches(cacheName)) { + LockManager lockManager = extractLockManager(cache); + getLog().tracef("Locks info=%s", lockManager.printLockInfo()); + if (lockManager.getNumberOfLocksHeld() != 0) { + getLog().warnf("Locks acquired on cache '%s'", cache); + return false; + } + } + return true; + }, 30000, 500, TimeUnit.MILLISECONDS); + } + + protected void assertValue(Object key, String value, Collection> caches) { + for (Cache cache : caches) { + AssertJUnit.assertEquals("Wrong value in cache " + address(cache), value, cache.get(key)); + } + } + + protected KeyInfo createKeys(String cacheName) { + final Object key1 = new MagicKey("k1", cache(1, cacheName), cache(2, cacheName)); + final Object key2 = new MagicKey("k2", cache(2, cacheName), cache(1, cacheName)); + cache(1, cacheName).put(key1, INITIAL_VALUE); + cache(2, cacheName).put(key2, INITIAL_VALUE); + return new KeyInfo(key1, key2); + } + + private void assertNoTransactionsInPartitionHandler(final String cacheName) { + eventually("Transactions pending in PartitionHandlingManager", () -> { + for (Cache cache : caches(cacheName)) { + Collection partialTransactions = extractComponent(cache, PartitionHandlingManager.class).getPartialTransactions(); + if (!partialTransactions.isEmpty()) { + getLog().debugf("transactions not finished in %s. %s", address(cache), partialTransactions); + return false; + } + } + return true; + }); + } + + protected enum SplitMode { + ORIGINATOR_ISOLATED { + @Override + public void split(BaseTxPartitionAndMergeTest test) { + test.getLog().debug("Splitting cluster isolating the originator."); + test.splitCluster(new int[]{0}, new int[]{1, 2, 3}); + test.getLog().debug("Cluster split."); + } + }, + BOTH_DEGRADED { + @Override + public void split(BaseTxPartitionAndMergeTest test) { + test.getLog().debug("Splitting cluster in equal partition"); + test.splitCluster(new int[]{0, 1}, new int[]{2, 3}); + test.getLog().debug("Cluster split."); + } + }, + PRIMARY_OWNER_ISOLATED { + @Override + public void split(BaseTxPartitionAndMergeTest test) { + test.getLog().debug("Splitting cluster isolating a primary owner."); + test.splitCluster(new int[]{2}, new int[]{0, 1, 3}); + test.getLog().debug("Cluster split."); + } + }; + + public abstract void split(BaseTxPartitionAndMergeTest test); + } + + private interface AwaitAndUnblock { + void await(long timeout, TimeUnit timeUnit) throws InterruptedException; + + void unblock(); + } + + private interface Filter extends AwaitAndUnblock { + boolean before(CacheRpcCommand command, Reply reply, DeliverOrder order); + } + + private static class ControlledInboundHandler implements PerCacheInboundInvocationHandler { + + private final PerCacheInboundInvocationHandler delegate; + private volatile Filter filter; + + private ControlledInboundHandler(PerCacheInboundInvocationHandler delegate) { + this.delegate = delegate; + } + + @Override + public void handle(CacheRpcCommand command, Reply reply, DeliverOrder order) { + final Filter currentFilter = filter; + if (currentFilter != null && currentFilter.before(command, reply, order)) { + delegate.handle(command, reply, order); + } + } + } + + private static class BlockingFilter implements Filter { + + private final Class aClass; + private final ReclosableLatch notifier; + private final ReclosableLatch blocker; + + private BlockingFilter(Class aClass) { + this.aClass = aClass; + blocker = new ReclosableLatch(false); + notifier = new ReclosableLatch(false); + } + + @Override + public boolean before(CacheRpcCommand command, Reply reply, DeliverOrder order) { + if (aClass.isAssignableFrom(command.getClass())) { + notifier.open(); + try { + blocker.await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + return true; + } + + public void await(long timeout, TimeUnit timeUnit) throws InterruptedException { + if (!notifier.await(timeout, timeUnit)) { + throw new TimeoutException(); + } + } + + public void unblock() { + blocker.open(); + } + } + + private static class NotifierFilter implements Filter { + + private final Class aClass; + private final CountDownLatch notifier; + + private NotifierFilter(Class aClass) { + this.aClass = aClass; + notifier = new CountDownLatch(1); + } + + @Override + public boolean before(CacheRpcCommand command, Reply reply, DeliverOrder order) { + if (aClass.isAssignableFrom(command.getClass())) { + notifier.countDown(); + } + return true; + } + + public void await(long timeout, TimeUnit timeUnit) throws InterruptedException { + if (!notifier.await(timeout, timeUnit)) { + throw new TimeoutException(); + } + } + + @Override + public void unblock() { + /*no-op*/ + } + } + + private static class DiscardFilter implements Filter { + + private final Class aClass; + private final ReclosableLatch notifier; + + private DiscardFilter(Class aClass) { + this.aClass = aClass; + notifier = new ReclosableLatch(false); + } + + @Override + public boolean before(CacheRpcCommand command, Reply reply, DeliverOrder order) { + if (!notifier.isOpened() && aClass.isAssignableFrom(command.getClass())) { + notifier.open(); + return false; + } + return true; + } + + public void await(long timeout, TimeUnit timeUnit) throws InterruptedException { + if (!notifier.await(timeout, timeUnit)) { + throw new TimeoutException(); + } + } + + @Override + public void unblock() { + /*no-op*/ + } + } + + protected static class KeyInfo { + private final Object key1; + private final Object key2; + + public KeyInfo(Object key1, Object key2) { + this.key1 = key1; + this.key2 = key2; + } + + public void putFinalValue(Cache cache) { + cache.put(key1, FINAL_VALUE); + cache.put(key2, FINAL_VALUE); + } + + public Object getKey1() { + return key1; + } + + public Object getKey2() { + return key2; + } + } + + protected static class FilterCollection implements AwaitAndUnblock { + private final Collection collection; + + public FilterCollection(Collection collection) { + this.collection = collection; + } + + @Override + public void await(long timeout, TimeUnit timeUnit) throws InterruptedException { + for (AwaitAndUnblock await : collection) { + await.await(timeout, timeUnit); + } + } + + public void unblock() { + collection.forEach(BaseTxPartitionAndMergeTest.AwaitAndUnblock::unblock); + } + } +} diff --git a/core/src/test/java/org/infinispan/partitionhandling/OptimisticTxPartitionAndMergeDuringCommitTest.java b/core/src/test/java/org/infinispan/partitionhandling/OptimisticTxPartitionAndMergeDuringCommitTest.java new file mode 100644 index 00000000000..14dd4242f89 --- /dev/null +++ b/core/src/test/java/org/infinispan/partitionhandling/OptimisticTxPartitionAndMergeDuringCommitTest.java @@ -0,0 +1,94 @@ +package org.infinispan.partitionhandling; + +import org.infinispan.Cache; +import org.infinispan.commands.tx.CommitCommand; +import org.infinispan.commands.tx.TransactionBoundaryCommand; +import org.infinispan.transaction.tm.DummyTransaction; +import org.infinispan.transaction.tm.DummyTransactionManager; +import org.infinispan.util.logging.Log; +import org.infinispan.util.logging.LogFactory; +import org.testng.AssertJUnit; +import org.testng.annotations.Test; + +/** + * It tests multiple scenarios where a split can happen during a transaction. + * + * @author Pedro Ruivo + * @since 8.0 + */ +@Test(groups = "functional", testName = "partitionhandling.OptimisticTxPartitionAndMergeDuringCommitTest") +public class OptimisticTxPartitionAndMergeDuringCommitTest extends BaseOptimisticTxPartitionAndMergeTest { + + private static final Log log = LogFactory.getLog(OptimisticTxPartitionAndMergeDuringCommitTest.class); + + public void testDegradedPartitionWithDiscard() throws Exception { + doTest(SplitMode.BOTH_DEGRADED, false, true); + } + + public void testDegradedPartition() throws Exception { + doTest(SplitMode.BOTH_DEGRADED, false, false); + } + + public void testOriginatorIsolatedPartitionWithDiscard() throws Exception { + doTest(SplitMode.ORIGINATOR_ISOLATED, false, true); + } + + public void testOriginatorIsolatedPartition() throws Exception { + doTest(SplitMode.ORIGINATOR_ISOLATED, false, false); + } + + public void testPrimaryOwnerIsolatedPartitionWithDiscard() throws Exception { + doTest(SplitMode.PRIMARY_OWNER_ISOLATED, false, true); + } + + public void testPrimaryOwnerIsolatedPartition() throws Exception { + doTest(SplitMode.PRIMARY_OWNER_ISOLATED, false, false); + } + + public void testSplitBeforeCommit() throws Exception { + //the transaction is successfully prepare and then the split happens before the commit phase starts. + waitForClusterToForm(OPTIMISTIC_TX_CACHE_NAME); + final KeyInfo keyInfo = createKeys(OPTIMISTIC_TX_CACHE_NAME); + final Cache originator = cache(0, OPTIMISTIC_TX_CACHE_NAME); + + final DummyTransactionManager transactionManager = (DummyTransactionManager) originator.getAdvancedCache().getTransactionManager(); + transactionManager.begin(); + final DummyTransaction transaction = transactionManager.getTransaction(); + keyInfo.putFinalValue(originator); + AssertJUnit.assertTrue(transaction.runPrepare()); + transactionManager.suspend(); + + SplitMode.BOTH_DEGRADED.split(this); + + transactionManager.resume(transaction); + transaction.runCommit(false); + + assertLocked(cache(1, OPTIMISTIC_TX_CACHE_NAME), keyInfo.getKey1()); + assertLocked(cache(2, OPTIMISTIC_TX_CACHE_NAME), keyInfo.getKey2()); + + mergeCluster(); + finalAsserts(OPTIMISTIC_TX_CACHE_NAME, keyInfo, FINAL_VALUE); + } + + @Override + protected void checkLocksDuringPartition(SplitMode splitMode, KeyInfo keyInfo, boolean discard) { + //on both caches, the key is locked and it is unlocked after the merge + assertLocked(cache(1, OPTIMISTIC_TX_CACHE_NAME), keyInfo.getKey1()); + assertLocked(cache(2, OPTIMISTIC_TX_CACHE_NAME), keyInfo.getKey2()); + } + + @Override + protected boolean forceRollback() { + return false; + } + + @Override + protected Class getCommandClass() { + return CommitCommand.class; + } + + @Override + protected Log getLog() { + return log; + } +} diff --git a/core/src/test/java/org/infinispan/partitionhandling/OptimisticTxPartitionAndMergeDuringPrepareTest.java b/core/src/test/java/org/infinispan/partitionhandling/OptimisticTxPartitionAndMergeDuringPrepareTest.java new file mode 100644 index 00000000000..71d1d6d6d2e --- /dev/null +++ b/core/src/test/java/org/infinispan/partitionhandling/OptimisticTxPartitionAndMergeDuringPrepareTest.java @@ -0,0 +1,78 @@ +package org.infinispan.partitionhandling; + +import org.infinispan.commands.tx.PrepareCommand; +import org.infinispan.commands.tx.TransactionBoundaryCommand; +import org.infinispan.util.logging.Log; +import org.infinispan.util.logging.LogFactory; +import org.testng.annotations.Test; + +/** + * It tests multiple scenarios where a split can happen during a transaction. + * + * @author Pedro Ruivo + * @since 8.0 + */ +@Test(groups = "functional", testName = "partitionhandling.OptimisticTxPartitionAndMergeDuringPrepareTest") +public class OptimisticTxPartitionAndMergeDuringPrepareTest extends BaseOptimisticTxPartitionAndMergeTest { + + private static final Log log = LogFactory.getLog(OptimisticTxPartitionAndMergeDuringPrepareTest.class); + + public void testDegradedPartitionWithDiscard() throws Exception { + doTest(SplitMode.BOTH_DEGRADED, true, true); + } + + public void testDegradedPartition() throws Exception { + doTest(SplitMode.BOTH_DEGRADED, true, false); + } + + public void testOriginatorIsolatedPartitionWithDiscard() throws Exception { + doTest(SplitMode.ORIGINATOR_ISOLATED, true, true); + } + + public void testOriginatorIsolatedPartition() throws Exception { + doTest(SplitMode.ORIGINATOR_ISOLATED, true, false); + } + + public void testPrimaryOwnerIsolatedPartitionWithDiscard() throws Exception { + doTest(SplitMode.PRIMARY_OWNER_ISOLATED, false, true); + } + + public void testPrimaryOwnerIsolatedPartition() throws Exception { + doTest(SplitMode.PRIMARY_OWNER_ISOLATED, false, false); + } + + @Override + protected void checkLocksDuringPartition(SplitMode splitMode, KeyInfo keyInfo, boolean discard) { + switch (splitMode) { + case ORIGINATOR_ISOLATED: + //they assume that the originator has crashed, so the prepare is never processed. + assertEventuallyNotLocked(cache(1, OPTIMISTIC_TX_CACHE_NAME), keyInfo.getKey1()); + break; + case PRIMARY_OWNER_ISOLATED: + //the originator can recover and will retry the prepare command until it succeeds. + assertEventuallyNotLocked(cache(1, OPTIMISTIC_TX_CACHE_NAME), keyInfo.getKey1()); + break; + case BOTH_DEGRADED: + //with the new changes, the rollback succeeds on the originator partition. Cache1 releases the lock. + assertEventuallyNotLocked(cache(1, OPTIMISTIC_TX_CACHE_NAME), keyInfo.getKey1()); + break; + } + //or the prepare is never received, so key never locked, or it is received and it decides to rollback the transaction. + assertEventuallyNotLocked(cache(2, OPTIMISTIC_TX_CACHE_NAME), keyInfo.getKey2()); + } + + @Override + protected boolean forceRollback() { + return false; + } + + @Override + protected Class getCommandClass() { + return PrepareCommand.class; + } + + @Override + protected Log getLog() { + return log; + } +} diff --git a/core/src/test/java/org/infinispan/partitionhandling/OptimisticTxPartitionAndMergeDuringRollbackTest.java b/core/src/test/java/org/infinispan/partitionhandling/OptimisticTxPartitionAndMergeDuringRollbackTest.java new file mode 100644 index 00000000000..7dd948910b1 --- /dev/null +++ b/core/src/test/java/org/infinispan/partitionhandling/OptimisticTxPartitionAndMergeDuringRollbackTest.java @@ -0,0 +1,105 @@ +package org.infinispan.partitionhandling; + +import org.infinispan.Cache; +import org.infinispan.commands.tx.RollbackCommand; +import org.infinispan.commands.tx.TransactionBoundaryCommand; +import org.infinispan.transaction.tm.DummyTransaction; +import org.infinispan.transaction.tm.DummyTransactionManager; +import org.infinispan.util.logging.Log; +import org.infinispan.util.logging.LogFactory; +import org.testng.AssertJUnit; +import org.testng.annotations.Test; + +/** + * It tests multiple scenarios where a split can happen during a transaction. + * + * @author Pedro Ruivo + * @since 8.0 + */ +@Test(groups = "functional", testName = "partitionhandling.OptimisticTxPartitionAndMergeDuringRollbackTest") +public class OptimisticTxPartitionAndMergeDuringRollbackTest extends BaseOptimisticTxPartitionAndMergeTest { + + private static final Log log = LogFactory.getLog(OptimisticTxPartitionAndMergeDuringRollbackTest.class); + + public void testDegradedPartitionWithDiscard() throws Exception { + doTest(SplitMode.BOTH_DEGRADED, true, true); + } + + public void testDegradedPartition() throws Exception { + doTest(SplitMode.BOTH_DEGRADED, true, false); + } + + public void testOriginatorIsolatedPartitionWithDiscard() throws Exception { + doTest(SplitMode.ORIGINATOR_ISOLATED, true, true); + } + + public void testOriginatorIsolatedPartition() throws Exception { + doTest(SplitMode.ORIGINATOR_ISOLATED, true, false); + } + + public void testPrimaryOwnerIsolatedPartitionWithDiscard() throws Exception { + doTest(SplitMode.PRIMARY_OWNER_ISOLATED, true, true); + } + + public void testPrimaryOwnerIsolatedPartition() throws Exception { + doTest(SplitMode.PRIMARY_OWNER_ISOLATED, true, false); + } + + public void testSplitBeforeRollback() throws Exception { + //the transaction is successfully prepare and then the split happens before the commit phase starts. + waitForClusterToForm(OPTIMISTIC_TX_CACHE_NAME); + final KeyInfo keyInfo = createKeys(OPTIMISTIC_TX_CACHE_NAME); + final Cache originator = cache(0, OPTIMISTIC_TX_CACHE_NAME); + + final DummyTransactionManager transactionManager = (DummyTransactionManager) originator.getAdvancedCache().getTransactionManager(); + transactionManager.begin(); + final DummyTransaction transaction = transactionManager.getTransaction(); + keyInfo.putFinalValue(originator); + AssertJUnit.assertTrue(transaction.runPrepare()); + transactionManager.suspend(); + + SplitMode.BOTH_DEGRADED.split(this); + + transactionManager.resume(transaction); + transaction.runCommit(true); + + assertEventuallyNotLocked(cache(1, OPTIMISTIC_TX_CACHE_NAME), keyInfo.getKey1()); + assertLocked(cache(2, OPTIMISTIC_TX_CACHE_NAME), keyInfo.getKey2()); + + mergeCluster(); + finalAsserts(OPTIMISTIC_TX_CACHE_NAME, keyInfo, INITIAL_VALUE); + } + + @Override + protected void checkLocksDuringPartition(SplitMode splitMode, KeyInfo keyInfo, boolean discard) { + if (splitMode == SplitMode.ORIGINATOR_ISOLATED && discard) { + //rollback never received, so key is locked until the merge occurs. + assertLocked(cache(1, OPTIMISTIC_TX_CACHE_NAME), keyInfo.getKey1()); + } else { + //key is unlocked because the rollback is always received in cache1 + assertEventuallyNotLocked(cache(1, OPTIMISTIC_TX_CACHE_NAME), keyInfo.getKey1()); + } + if (discard) { + //rollback never received, so key is locked until the merge occurs. + assertLocked(cache(2, OPTIMISTIC_TX_CACHE_NAME), keyInfo.getKey2()); + } else { + //rollback received, so key is unlocked + assertEventuallyNotLocked(cache(2, OPTIMISTIC_TX_CACHE_NAME), keyInfo.getKey2()); + } + } + + @Override + protected boolean forceRollback() { + return true; + } + + @Override + protected Class getCommandClass() { + return RollbackCommand.class; + } + + @Override + protected Log getLog() { + return log; + } +} diff --git a/core/src/test/java/org/infinispan/partitionhandling/PessimisticTxPartitionAndMergeDuringPrepareTest.java b/core/src/test/java/org/infinispan/partitionhandling/PessimisticTxPartitionAndMergeDuringPrepareTest.java new file mode 100644 index 00000000000..6b96a0a468d --- /dev/null +++ b/core/src/test/java/org/infinispan/partitionhandling/PessimisticTxPartitionAndMergeDuringPrepareTest.java @@ -0,0 +1,99 @@ +package org.infinispan.partitionhandling; + +import org.infinispan.Cache; +import org.infinispan.commands.tx.PrepareCommand; +import org.infinispan.commands.tx.TransactionBoundaryCommand; +import org.infinispan.util.logging.Log; +import org.infinispan.util.logging.LogFactory; +import org.testng.annotations.Test; + +import javax.transaction.Transaction; +import javax.transaction.TransactionManager; + +/** + * It tests multiple scenarios where a split can happen during a transaction. + * + * @author Pedro Ruivo + * @since 8.0 + */ +@Test(groups = "functional", testName = "partitionhandling.PessimisticTxPartitionAndMergeDuringPrepareTest") +public class PessimisticTxPartitionAndMergeDuringPrepareTest extends BasePessimisticTxPartitionAndMergeTest { + + private static final Log log = LogFactory.getLog(PessimisticTxPartitionAndMergeDuringPrepareTest.class); + + public void testDegradedPartitionWithDiscard() throws Exception { + doTest(SplitMode.BOTH_DEGRADED, false, true); + } + + public void testDegradedPartition() throws Exception { + doTest(SplitMode.BOTH_DEGRADED, false, false); + } + + public void testOriginatorIsolatedPartitionWithDiscard() throws Exception { + doTest(SplitMode.ORIGINATOR_ISOLATED, false, true); + } + + public void testOriginatorIsolatedPartition() throws Exception { + doTest(SplitMode.ORIGINATOR_ISOLATED, false, false); + } + + public void testPrimaryOwnerIsolatedPartitionWithDiscard() throws Exception { + doTest(SplitMode.PRIMARY_OWNER_ISOLATED, false, true); + } + + public void testPrimaryOwnerIsolatedPartition() throws Exception { + doTest(SplitMode.PRIMARY_OWNER_ISOLATED, false, false); + } + + public void testSplitBeforePrepare() throws Exception { + //split happens before the commit() or rollback(). Locks are acquired + waitForClusterToForm(PESSIMISTIC_TX_CACHE_NAME); + + final KeyInfo keyInfo = createKeys(PESSIMISTIC_TX_CACHE_NAME); + final Cache originator = cache(0, PESSIMISTIC_TX_CACHE_NAME); + + final TransactionManager transactionManager = originator.getAdvancedCache().getTransactionManager(); + transactionManager.begin(); + keyInfo.putFinalValue(originator); + final Transaction transaction = transactionManager.suspend(); + + SplitMode.BOTH_DEGRADED.split(this); + + transactionManager.resume(transaction); + transactionManager.commit(); + + assertLocked(cache(1, PESSIMISTIC_TX_CACHE_NAME), keyInfo.getKey1()); + assertLocked(cache(2, PESSIMISTIC_TX_CACHE_NAME), keyInfo.getKey2()); + + mergeCluster(); + finalAsserts(PESSIMISTIC_TX_CACHE_NAME, keyInfo, FINAL_VALUE); + } + + @Override + protected void checkLocksDuringPartition(SplitMode splitMode, KeyInfo keyInfo, boolean discard) { + //always locked: locks acquired in runtime + assertLocked(cache(1, PESSIMISTIC_TX_CACHE_NAME), keyInfo.getKey1()); + if (discard) { + //locks are acquired during runtime + assertLocked(cache(2, PESSIMISTIC_TX_CACHE_NAME), keyInfo.getKey2()); + } else { + //prepare will succeed, but the locks are not released (the TxCompletionNotificationCommand will do it) + assertLocked(cache(2, PESSIMISTIC_TX_CACHE_NAME), keyInfo.getKey2()); + } + } + + @Override + protected boolean forceRollback() { + return false; + } + + @Override + protected Class getCommandClass() { + return PrepareCommand.class; + } + + @Override + protected Log getLog() { + return log; + } +} diff --git a/core/src/test/java/org/infinispan/partitionhandling/PessimisticTxPartitionAndMergeDuringRollbackTest.java b/core/src/test/java/org/infinispan/partitionhandling/PessimisticTxPartitionAndMergeDuringRollbackTest.java new file mode 100644 index 00000000000..880bee1bb20 --- /dev/null +++ b/core/src/test/java/org/infinispan/partitionhandling/PessimisticTxPartitionAndMergeDuringRollbackTest.java @@ -0,0 +1,109 @@ +package org.infinispan.partitionhandling; + +import org.infinispan.Cache; +import org.infinispan.commands.tx.RollbackCommand; +import org.infinispan.commands.tx.TransactionBoundaryCommand; +import org.infinispan.util.logging.Log; +import org.infinispan.util.logging.LogFactory; +import org.testng.annotations.Test; + +import javax.transaction.Transaction; +import javax.transaction.TransactionManager; + +/** + * It tests multiple scenarios where a split can happen during a transaction. + * + * @author Pedro Ruivo + * @since 8.0 + */ +@Test(groups = "functional", testName = "partitionhandling.PessimisticTxPartitionAndMergeDuringRollbackTest") +public class PessimisticTxPartitionAndMergeDuringRollbackTest extends BasePessimisticTxPartitionAndMergeTest { + + private static final Log log = LogFactory.getLog(PessimisticTxPartitionAndMergeDuringRollbackTest.class); + + public void testDegradedPartitionWithDiscard() throws Exception { + doTest(SplitMode.BOTH_DEGRADED, true, true); + } + + public void testDegradedPartition() throws Exception { + doTest(SplitMode.BOTH_DEGRADED, true, false); + } + + public void testOriginatorIsolatedPartitionWithDiscard() throws Exception { + doTest(SplitMode.ORIGINATOR_ISOLATED, true, true); + } + + public void testOriginatorIsolatedPartition() throws Exception { + doTest(SplitMode.ORIGINATOR_ISOLATED, true, false); + } + + public void testPrimaryOwnerIsolatedPartitionWithDiscard() throws Exception { + doTest(SplitMode.PRIMARY_OWNER_ISOLATED, true, true); + } + + public void testPrimaryOwnerIsolatedPartition() throws Exception { + doTest(SplitMode.PRIMARY_OWNER_ISOLATED, true, false); + } + + public void testSplitBeforeRollback() throws Exception { + //split happens before the commit() or rollback(). Locks are acquired + waitForClusterToForm(PESSIMISTIC_TX_CACHE_NAME); + + final KeyInfo keyInfo = createKeys(PESSIMISTIC_TX_CACHE_NAME); + final Cache originator = cache(0, PESSIMISTIC_TX_CACHE_NAME); + + final TransactionManager transactionManager = originator.getAdvancedCache().getTransactionManager(); + transactionManager.begin(); + keyInfo.putFinalValue(originator); + final Transaction transaction = transactionManager.suspend(); + + SplitMode.BOTH_DEGRADED.split(this); + + transactionManager.resume(transaction); + transactionManager.rollback(); + + assertEventuallyNotLocked(cache(1, PESSIMISTIC_TX_CACHE_NAME), keyInfo.getKey1()); + assertLocked(cache(2, PESSIMISTIC_TX_CACHE_NAME), keyInfo.getKey2()); + + mergeCluster(); + finalAsserts(PESSIMISTIC_TX_CACHE_NAME, keyInfo, INITIAL_VALUE); + } + + @Override + protected void checkLocksDuringPartition(SplitMode splitMode, KeyInfo keyInfo, boolean discard) { + if (splitMode == SplitMode.ORIGINATOR_ISOLATED) { + if (discard) { + //rollback is never received. + assertLocked(cache(1, PESSIMISTIC_TX_CACHE_NAME), keyInfo.getKey1()); + } else { + //rollback can be delivered + assertEventuallyNotLocked(cache(1, PESSIMISTIC_TX_CACHE_NAME), keyInfo.getKey1()); + } + } else { + //key is unlocked because the rollback is always received in cache1 + assertEventuallyNotLocked(cache(1, PESSIMISTIC_TX_CACHE_NAME), keyInfo.getKey1()); + } + if (discard) { + //rollback never received, so key is locked until the merge occurs. + assertLocked(cache(2, PESSIMISTIC_TX_CACHE_NAME), keyInfo.getKey2()); + } else { + //rollback received, so key is unlocked + assertEventuallyNotLocked(cache(2, PESSIMISTIC_TX_CACHE_NAME), keyInfo.getKey2()); + } + } + + @Override + protected boolean forceRollback() { + return true; + } + + @Override + protected Class getCommandClass() { + return RollbackCommand.class; + } + + @Override + protected Log getLog() { + return log; + } +} diff --git a/core/src/test/java/org/infinispan/partitionhandling/PessimisticTxPartitionAndMergeDuringRuntimeTest.java b/core/src/test/java/org/infinispan/partitionhandling/PessimisticTxPartitionAndMergeDuringRuntimeTest.java new file mode 100644 index 00000000000..11b168cb548 --- /dev/null +++ b/core/src/test/java/org/infinispan/partitionhandling/PessimisticTxPartitionAndMergeDuringRuntimeTest.java @@ -0,0 +1,76 @@ +package org.infinispan.partitionhandling; + +import org.infinispan.commands.control.LockControlCommand; +import org.infinispan.commands.tx.TransactionBoundaryCommand; +import org.infinispan.util.logging.Log; +import org.infinispan.util.logging.LogFactory; +import org.testng.annotations.Test; + +/** + * It tests multiple scenarios where a split can happen during a transaction. + * + * @author Pedro Ruivo + * @since 8.0 + */ +@Test(groups = "functional", testName = "partitionhandling.PessimisticTxPartitionAndMergeDuringRuntimeTest") +public class PessimisticTxPartitionAndMergeDuringRuntimeTest extends BasePessimisticTxPartitionAndMergeTest { + + private static final Log log = LogFactory.getLog(PessimisticTxPartitionAndMergeDuringRuntimeTest.class); + + public void testDegradedPartitionWithDiscard() throws Exception { + doTest(SplitMode.BOTH_DEGRADED, true, true); + } + + public void testDegradedPartition() throws Exception { + doTest(SplitMode.BOTH_DEGRADED, true, false); + } + + public void testOriginatorIsolatedPartitionWithDiscard() throws Exception { + doTest(SplitMode.ORIGINATOR_ISOLATED, true, true); + } + + public void testOriginatorIsolatedPartition() throws Exception { + doTest(SplitMode.ORIGINATOR_ISOLATED, true, false); + } + + public void testPrimaryOwnerIsolatedPartitionWithDiscard() throws Exception { + doTest(SplitMode.PRIMARY_OWNER_ISOLATED, false, true); + } + + public void testPrimaryOwnerIsolatedPartition() throws Exception { + doTest(SplitMode.PRIMARY_OWNER_ISOLATED, false, false); + } + + @Override + protected void checkLocksDuringPartition(SplitMode splitMode, KeyInfo keyInfo, boolean discard) { + switch (splitMode) { + case ORIGINATOR_ISOLATED: + //lock command never received or ignored because the originator is in another partition + //(safe because we will rollback anyway) + assertEventuallyNotLocked(cache(1, PESSIMISTIC_TX_CACHE_NAME), keyInfo.getKey1()); + case BOTH_DEGRADED: + //originator rollbacks which is received by cache1 + assertEventuallyNotLocked(cache(1, PESSIMISTIC_TX_CACHE_NAME), keyInfo.getKey1()); + case PRIMARY_OWNER_ISOLATED: + //originator can commit the transaction, key is eventually unlocked + assertEventuallyNotLocked(cache(1, PESSIMISTIC_TX_CACHE_NAME), keyInfo.getKey1()); + } + //lock command is discarded since the transaction originator is missing + assertEventuallyNotLocked(cache(2, PESSIMISTIC_TX_CACHE_NAME), keyInfo.getKey2()); + } + + @Override + protected boolean forceRollback() { + return false; + } + + @Override + protected Class getCommandClass() { + return LockControlCommand.class; + } + + @Override + protected Log getLog() { + return log; + } +} diff --git a/core/src/test/java/org/infinispan/replication/AsyncReplTest.java b/core/src/test/java/org/infinispan/replication/AsyncReplTest.java index 21eba4f2b4a..039fff1054f 100644 --- a/core/src/test/java/org/infinispan/replication/AsyncReplTest.java +++ b/core/src/test/java/org/infinispan/replication/AsyncReplTest.java @@ -22,8 +22,8 @@ protected void createCacheManagers() throws Throwable { public void testWithNoTx() throws Exception { - Cache cache1 = cache(0,"asyncRepl"); - Cache cache2 = cache(1,"asyncRepl"); + Cache cache1 = cache(0,"asyncRepl"); + Cache cache2 = cache(1,"asyncRepl"); String key = "key"; replListener(cache2).expect(PutKeyValueCommand.class); @@ -44,15 +44,15 @@ public void testWithNoTx() throws Exception { } public void testWithTx() throws Exception { - Cache cache1 = cache(0,"asyncRepl"); - Cache cache2 = cache(1,"asyncRepl"); + Cache cache1 = cache(0,"asyncRepl"); + Cache cache2 = cache(1,"asyncRepl"); String key = "key"; replListener(cache2).expect(PutKeyValueCommand.class); cache1.put(key, "value1"); // allow for replication replListener(cache2).waitForRpc(); - assertNotLocked(cache1, key); + assertEventuallyNotLocked(cache1, key); assertEquals("value1", cache1.get(key)); assertEquals("value1", cache2.get(key)); @@ -65,7 +65,7 @@ public void testWithTx() throws Exception { assertEquals("value1", cache2.get(key)); mgr.commit(); replListener(cache2).waitForRpc(); - assertNotLocked(cache1, key); + assertEventuallyNotLocked(cache1, key); assertEquals("value2", cache1.get(key)); assertEquals("value2", cache2.get(key)); @@ -80,13 +80,13 @@ public void testWithTx() throws Exception { assertEquals("value2", cache1.get(key)); assertEquals("value2", cache2.get(key)); - assertNotLocked(cache1, key); + assertEventuallyNotLocked(cache1, key); } public void simpleTest() throws Exception { - Cache cache1 = cache(0,"asyncRepl"); - Cache cache2 = cache(1,"asyncRepl"); + Cache cache1 = cache(0,"asyncRepl"); + cache(1, "asyncRepl"); String key = "key"; TransactionManager mgr = TestingUtil.getTransactionManager(cache1); @@ -95,7 +95,7 @@ public void simpleTest() throws Exception { cache1.put(key, "value3"); mgr.rollback(); - assertNotLocked(cache1, key); + assertEventuallyNotLocked(cache1, key); } } diff --git a/core/src/test/java/org/infinispan/replication/SyncReplLockingTest.java b/core/src/test/java/org/infinispan/replication/SyncReplLockingTest.java index 477c7e5e18a..7f034eef21d 100644 --- a/core/src/test/java/org/infinispan/replication/SyncReplLockingTest.java +++ b/core/src/test/java/org/infinispan/replication/SyncReplLockingTest.java @@ -81,8 +81,8 @@ public void testLocksReleasedWithNoMods() throws Exception { cache1.get(k); mgr.commit(); - assertNotLocked(cache1, "testcache"); - assertNotLocked(cache2, "testcache"); + assertEventuallyNotLocked(cache1, "testcache"); + assertEventuallyNotLocked(cache2, "testcache"); assert cache1.isEmpty(); assert cache2.isEmpty(); @@ -111,8 +111,8 @@ public void testReplaceNonExistentKey() throws Exception { assertNull("Should be null", cache1.get(k)); mgr.commit(); - assertNotLocked(cache1, "testcache"); - assertNotLocked(cache2, "testcache"); + assertEventuallyNotLocked(cache1, "testcache"); + assertEventuallyNotLocked(cache2, "testcache"); assert cache1.isEmpty(); assert cache2.isEmpty(); diff --git a/core/src/test/java/org/infinispan/statetransfer/StaleLocksWithCommitDuringStateTransferTest.java b/core/src/test/java/org/infinispan/statetransfer/StaleLocksWithCommitDuringStateTransferTest.java index c379abc5a02..ee679d0b6d4 100644 --- a/core/src/test/java/org/infinispan/statetransfer/StaleLocksWithCommitDuringStateTransferTest.java +++ b/core/src/test/java/org/infinispan/statetransfer/StaleLocksWithCommitDuringStateTransferTest.java @@ -124,10 +124,10 @@ public void run() { } // test that we don't leak locks - assertNotLocked(c1, k1); - assertNotLocked(c2, k1); - assertNotLocked(c1, k2); - assertNotLocked(c2, k2); + assertEventuallyNotLocked(c1, k1); + assertEventuallyNotLocked(c2, k1); + assertEventuallyNotLocked(c1, k2); + assertEventuallyNotLocked(c2, k2); } public void testRollbackSuspectFailure() throws Exception { @@ -201,7 +201,7 @@ public void run() { } // test that we don't leak locks - assertNotLocked(c1, k1); - assertNotLocked(c1, k2); + assertEventuallyNotLocked(c1, k1); + assertEventuallyNotLocked(c1, k2); } } diff --git a/core/src/test/java/org/infinispan/test/AbstractCacheTest.java b/core/src/test/java/org/infinispan/test/AbstractCacheTest.java index 2dd6a3e0e96..c30ef5c0214 100644 --- a/core/src/test/java/org/infinispan/test/AbstractCacheTest.java +++ b/core/src/test/java/org/infinispan/test/AbstractCacheTest.java @@ -7,7 +7,11 @@ import org.infinispan.test.fwk.CleanupAfterMethod; import org.infinispan.test.fwk.CleanupAfterTest; import org.infinispan.test.fwk.TestCacheManagerFactory; -import org.infinispan.util.concurrent.locks.LockManager; + +import java.util.concurrent.TimeUnit; + +import static java.lang.String.format; +import static org.testng.AssertJUnit.assertTrue; /** * Base class for {@link org.infinispan.test.SingleCacheManagerTest} and {@link org.infinispan.test.MultipleCacheManagersTest}. @@ -16,7 +20,7 @@ */ public class AbstractCacheTest extends AbstractInfinispanTest { - public static enum CleanupPhase { + public enum CleanupPhase { AFTER_METHOD, AFTER_TEST } @@ -70,23 +74,31 @@ protected boolean xor(boolean b1, boolean b2) { return (b1 || b2) && !(b1 && b2); } - protected void assertNotLocked(final Cache cache, final Object key) { + protected void assertEventuallyNotLocked(final Cache cache, final Object key) { //lock release happens async, hence the eventually... - eventually(new Condition() { + eventually(format("Expected key '%s' to be unlocked on cache '%s'", key, cache), new Condition() { @Override public boolean isSatisfied() throws Exception { return !checkLocked(cache, key); } - }); + }, 20000, 500, TimeUnit.MILLISECONDS); + } + + protected void assertEventuallyLocked(final Cache cache, final Object key) { + eventually(format("Expected key '%s' to be locked on cache '%s'", key, cache), new Condition() { + @Override + public boolean isSatisfied() throws Exception { + return checkLocked(cache, key); + } + }, 20000, 500, TimeUnit.MILLISECONDS); } protected void assertLocked(Cache cache, Object key) { - assert checkLocked(cache, key) : "expected key '" + key + "' to be locked on cache " + cache + ", but it is not"; + assertTrue(format("Expected key '%s' to be locked on cache '%s'", key, cache), checkLocked(cache, key)); } protected boolean checkLocked(Cache cache, Object key) { - LockManager lockManager = TestingUtil.extractLockManager(cache); - return lockManager.isLocked(key); + return TestingUtil.extractLockManager(cache).isLocked(key); } public EmbeddedCacheManager manager(Cache c) { diff --git a/core/src/test/java/org/infinispan/test/AbstractInfinispanTest.java b/core/src/test/java/org/infinispan/test/AbstractInfinispanTest.java index 460112f66d4..30cbcd7d343 100644 --- a/core/src/test/java/org/infinispan/test/AbstractInfinispanTest.java +++ b/core/src/test/java/org/infinispan/test/AbstractInfinispanTest.java @@ -78,18 +78,30 @@ protected void eventually(Condition ec, long timeoutMillis) { */ @Deprecated protected void eventually(Condition ec, long timeoutMillis, int loops) { + eventually(null, ec, timeoutMillis, loops); + } + + /** + * @deprecated Use {@link #eventually(String, Condition, long, long, TimeUnit)} instead. + */ + @Deprecated + protected void eventually(String message, Condition ec, long timeoutMillis, int loops) { if (loops <= 0) { throw new IllegalArgumentException("Number of loops must be positive"); } long sleepDuration = timeoutMillis / loops + 1; - eventually(ec, timeoutMillis, sleepDuration, TimeUnit.MILLISECONDS); + eventually(message, ec, timeoutMillis, sleepDuration, TimeUnit.MILLISECONDS); } protected void eventually(Condition ec, long timeout, TimeUnit unit) { - eventually(ec, timeout, 500, unit); + eventually(null, ec, timeout, 500, unit); } protected void eventually(Condition ec, long timeout, long pollInterval, TimeUnit unit) { + eventually(null, ec, timeout, pollInterval, unit); + } + + protected void eventually(String message, Condition ec, long timeout, long pollInterval, TimeUnit unit) { if (pollInterval <= 0) { throw new IllegalArgumentException("Check interval must be positive"); } @@ -100,7 +112,7 @@ protected void eventually(Condition ec, long timeout, long pollInterval, TimeUni if (ec.isSatisfied()) return; Thread.sleep(sleepMillis); } - assertTrue(ec.isSatisfied()); + assertTrue(message, ec.isSatisfied()); } catch (Exception e) { throw new RuntimeException("Unexpected!", e); } @@ -225,8 +237,8 @@ protected CompletionService runConcurrentlyWithCompletionService(Callable } CompletionService completionService = completionService(); CyclicBarrier barrier = new CyclicBarrier(tasks.length); - for (int i = 0; i < tasks.length; i++) { - completionService.submit(new ConcurrentCallable(new LoggingCallable(tasks[i]), barrier)); + for (Callable task : tasks) { + completionService.submit(new ConcurrentCallable<>(new LoggingCallable(task), barrier)); } return completionService; @@ -417,8 +429,12 @@ protected void eventually(Condition ec) { eventually(ec, 10000); } + protected void eventually(String message, Condition ec) { + eventually(message, ec, 10000, 500, TimeUnit.MILLISECONDS); + } + protected interface Condition { - public boolean isSatisfied() throws Exception; + boolean isSatisfied() throws Exception; } private class LoggingCallable implements Callable { diff --git a/core/src/test/java/org/infinispan/test/MultipleCacheManagersTest.java b/core/src/test/java/org/infinispan/test/MultipleCacheManagersTest.java index 98a2c617668..4853c7a2944 100644 --- a/core/src/test/java/org/infinispan/test/MultipleCacheManagersTest.java +++ b/core/src/test/java/org/infinispan/test/MultipleCacheManagersTest.java @@ -172,7 +172,7 @@ protected EmbeddedCacheManager addClusterEnabledCacheManager(GlobalConfiguration * Creates a new optionally transactional cache manager, starts it, and adds it to the list of known cache managers on * the current thread. Uses a default clustered cache manager global config. * - * @param defaultConfig default cfg to use + * @param builder default cfg to use * @return the new CacheManager */ protected EmbeddedCacheManager addClusterEnabledCacheManager(ConfigurationBuilder builder, TransportFlags flags) { @@ -187,13 +187,6 @@ protected EmbeddedCacheManager addClusterEnabledCacheManager(GlobalConfiguration return cm; } - /** - * Creates a new cache manager, starts it, and adds it to the list of known cache managers on the current thread. - * @param mode cache mode to use - * @param transactional if true, the configuration will be decorated with necessary transactional settings - * @return an embedded cache manager - */ - protected void createCluster(ConfigurationBuilder builder, int count) { for (int i = 0; i < count; i++) addClusterEnabledCacheManager(builder); } @@ -268,7 +261,7 @@ protected List> createClusteredCaches( protected List> createClusteredCaches( int numMembersInCluster, String cacheName, ConfigurationBuilder builder, TransportFlags flags) { - List> caches = new ArrayList>(numMembersInCluster); + List> caches = new ArrayList<>(numMembersInCluster); for (int i = 0; i < numMembersInCluster; i++) { EmbeddedCacheManager cm = addClusterEnabledCacheManager(flags); cm.defineConfiguration(cacheName, builder.build()); @@ -281,7 +274,7 @@ protected List> createClusteredCaches( protected List> createClusteredCaches(int numMembersInCluster, ConfigurationBuilder defaultConfigBuilder) { - List> caches = new ArrayList>(numMembersInCluster); + List> caches = new ArrayList<>(numMembersInCluster); for (int i = 0; i < numMembersInCluster; i++) { EmbeddedCacheManager cm = addClusterEnabledCacheManager(defaultConfigBuilder); Cache cache = cm.getCache(); @@ -295,7 +288,7 @@ protected List> createClusteredCaches(int numMembersInCluster protected List> createClusteredCaches(int numMembersInCluster, ConfigurationBuilder defaultConfig, TransportFlags flags) { - List> caches = new ArrayList>(numMembersInCluster); + List> caches = new ArrayList<>(numMembersInCluster); for (int i = 0; i < numMembersInCluster; i++) { EmbeddedCacheManager cm = addClusterEnabledCacheManager(defaultConfig, flags); Cache cache = cm.getCache(); @@ -306,7 +299,7 @@ protected List> createClusteredCaches(int numMembersInCluster } /** - * Create cacheNames.lenght in each CacheManager (numMembersInCluster cacheManagers). + * Create cacheNames.length in each CacheManager (numMembersInCluster cacheManagers). * * @param numMembersInCluster * @param defaultConfigBuilder @@ -315,10 +308,10 @@ protected List> createClusteredCaches(int numMembersInCluster */ protected List>> createClusteredCaches(int numMembersInCluster, ConfigurationBuilder defaultConfigBuilder, String[] cacheNames) { - List>> allCaches = new ArrayList>>(numMembersInCluster); + List>> allCaches = new ArrayList<>(numMembersInCluster); for (int i = 0; i < numMembersInCluster; i++) { EmbeddedCacheManager cm = addClusterEnabledCacheManager(defaultConfigBuilder); - List> currentCacheManagerCaches = new ArrayList>(cacheNames.length); + List> currentCacheManagerCaches = new ArrayList<>(cacheNames.length); for (String cacheName : cacheNames) { Cache cache = cm.getCache(cacheName); @@ -498,7 +491,7 @@ protected boolean checkTxCount(int cacheIndex, int localTx, int remoteTx) { } protected void assertNotLocked(int cacheIndex, Object key) { - assertNotLocked(cache(cacheIndex), key); + assertEventuallyNotLocked(cache(cacheIndex), key); } protected void assertLocked(int cacheIndex, Object key) { @@ -516,13 +509,13 @@ protected Cache getLockOwner(Object key) { protected Cache getLockOwner(Object key, String cacheName) { Configuration c = getCache(0, cacheName).getCacheConfiguration(); if (c.clustering().cacheMode().isReplicated() || c.clustering().cacheMode().isInvalidation()) { - return (Cache) getCache(0, cacheName); //for replicated caches only the coordinator acquires lock + return getCache(0, cacheName); //for replicated caches only the coordinator acquires lock } else { if (!c.clustering().cacheMode().isDistributed()) throw new IllegalStateException("This is not a clustered cache!"); final Address address = getCache(0, cacheName).getAdvancedCache().getDistributionManager().locate(key).get(0); - for (Cache cache : caches(cacheName)) { + for (Cache cache : this.caches(cacheName)) { if (cache.getAdvancedCache().getRpcManager().getTransport().getAddress().equals(address)) { - return (Cache) cache; + return cache; } } throw new IllegalStateException(); @@ -543,8 +536,8 @@ protected void assertKeyLockedCorrectly(Object key, String cacheName) { } } - private Cache getCache(int index, String name) { - return name == null ? cache(index) : cache(index, name); + private Cache getCache(int index, String name) { + return name == null ? this.cache(index) : this.cache(index, name); } protected void forceTwoPhase(int cacheIndex) throws SystemException, RollbackException { @@ -554,15 +547,20 @@ protected void forceTwoPhase(int cacheIndex) throws SystemException, RollbackExc } protected void assertNoTransactions() { - eventually(new Condition() { + assertNoTransactions(null); + } + + protected void assertNoTransactions(final String cacheName) { + eventually("There are pending transactions!", new Condition() { @Override public boolean isSatisfied() throws Exception { - for (int i = 0; i < caches().size(); i++) { - int localTxCount = transactionTable(i).getLocalTxCount(); - int remoteTxCount = transactionTable(i).getRemoteTxCount(); + for (Cache cache : caches(cacheName)) { + final TransactionTable transactionTable = TestingUtil.extractComponent(cache, TransactionTable.class); + int localTxCount = transactionTable.getLocalTxCount(); + int remoteTxCount = transactionTable.getRemoteTxCount(); if (localTxCount != 0 || remoteTxCount != 0) { - log.tracef("Local tx=%s, remote tx=%s, for cache %s ", - localTxCount, remoteTxCount, i); + log.tracef("Local tx=%s, remote tx=%s, for cache %s ", transactionTable.getLocalGlobalTransaction(), + transactionTable.getRemoteGlobalTransaction(), address(cache)); return false; } } @@ -582,7 +580,7 @@ protected void assertEventuallyEquals( @Override public boolean isSatisfied() throws Exception { return value == null - ? value == cache(cacheIndex).get(key) + ? null == cache(cacheIndex).get(key) : value.equals(cache(cacheIndex).get(key)); } }); diff --git a/core/src/test/java/org/infinispan/test/arquillian/DatagridManager.java b/core/src/test/java/org/infinispan/test/arquillian/DatagridManager.java index 7140274f190..d3bcc1c2070 100644 --- a/core/src/test/java/org/infinispan/test/arquillian/DatagridManager.java +++ b/core/src/test/java/org/infinispan/test/arquillian/DatagridManager.java @@ -79,7 +79,7 @@ public boolean xorOp(boolean b1, boolean b2) { //name change public void assertKeyNotLocked(Cache cache, Object key) { - assertNotLocked(cache, key); + assertEventuallyNotLocked(cache, key); } //name change diff --git a/core/src/test/java/org/infinispan/tx/StaleLockAfterTxAbortTest.java b/core/src/test/java/org/infinispan/tx/StaleLockAfterTxAbortTest.java index 9a3b016a681..a7de5a11f6e 100644 --- a/core/src/test/java/org/infinispan/tx/StaleLockAfterTxAbortTest.java +++ b/core/src/test/java/org/infinispan/tx/StaleLockAfterTxAbortTest.java @@ -37,7 +37,7 @@ protected EmbeddedCacheManager createCacheManager() throws Exception { public void doTest() throws Throwable { cache.put(k, "value"); // init value - assertNotLocked(cache, k); + assertEventuallyNotLocked(cache, k); // InvocationContextContainerImpl icc = (InvocationContextContainerImpl) TestingUtil.extractComponent(cache, InvocationContextContainer.class); // InvocationContext ctx = icc.getInvocationContext(); @@ -71,7 +71,7 @@ public void doTest() throws Throwable { transaction.runCommit(true); transactionThread.join(); - assertNotLocked(cache, k); + assertEventuallyNotLocked(cache, k); } private class TxThread extends Thread { diff --git a/core/src/test/java/org/infinispan/tx/TransactionXaAdapterTmIntegrationTest.java b/core/src/test/java/org/infinispan/tx/TransactionXaAdapterTmIntegrationTest.java index c134359742a..da9cc31b78c 100644 --- a/core/src/test/java/org/infinispan/tx/TransactionXaAdapterTmIntegrationTest.java +++ b/core/src/test/java/org/infinispan/tx/TransactionXaAdapterTmIntegrationTest.java @@ -35,9 +35,6 @@ */ @Test(testName = "tx.TransactionXaAdapterTmIntegrationTest", groups = "unstable", description = "Disabled due to instability - see ISPN-1123 -- original group: unit") public class TransactionXaAdapterTmIntegrationTest { - private Configuration configuration; - private XaTransactionTable txTable; - private GlobalTransaction globalTransaction; private LocalXaTransaction localTx; private TransactionXaAdapter xaAdapter; private DummyXid xid; @@ -47,15 +44,15 @@ public class TransactionXaAdapterTmIntegrationTest { @BeforeMethod public void setUp() throws XAException { Cache mockCache = mock(Cache.class); - configuration = new ConfigurationBuilder().build(); - txTable = new XaTransactionTable(); + Configuration configuration = new ConfigurationBuilder().build(); + XaTransactionTable txTable = new XaTransactionTable(); txTable.initialize(null, configuration, null, null, null, null, - null, null, null, null, mockCache, null, null); + null, null, null, null, mockCache, null, null, null); txTable.start(); txTable.startXidMapping(); TransactionFactory gtf = new TransactionFactory(); gtf.init(false, false, true, false); - globalTransaction = gtf.newGlobalTransaction(null, false); + GlobalTransaction globalTransaction = gtf.newGlobalTransaction(null, false); DummyBaseTransactionManager tm = new DummyBaseTransactionManager(); localTx = new LocalXaTransaction(new DummyTransaction(tm), globalTransaction, false, 1, AnyEquivalence.getInstance(), 0); xid = new DummyXid(uuid); @@ -66,7 +63,7 @@ public void setUp() throws XAException { txCoordinator = new TransactionCoordinator(); txCoordinator.init(commandsFactory, icf, invoker, txTable, null, configuration); xaAdapter = new TransactionXaAdapter(localTx, txTable, null, txCoordinator, null, null, - new ClusteringDependentLogic.InvalidationLogic(), configuration, ""); + new ClusteringDependentLogic.InvalidationLogic(), configuration, "", null); xaAdapter.start(xid, 0); } diff --git a/core/src/test/java/org/infinispan/tx/recovery/PostCommitRecoveryStateTest.java b/core/src/test/java/org/infinispan/tx/recovery/PostCommitRecoveryStateTest.java index 8a23de28e6e..c86bf2976da 100644 --- a/core/src/test/java/org/infinispan/tx/recovery/PostCommitRecoveryStateTest.java +++ b/core/src/test/java/org/infinispan/tx/recovery/PostCommitRecoveryStateTest.java @@ -89,11 +89,11 @@ public RecoveryIterator getPreparedTransactionsFromCluster() { } @Override - public void removeRecoveryInformationFromCluster(Collection
where, Xid xid, boolean sync, GlobalTransaction gtx) { + public void removeRecoveryInformation(Collection
where, Xid xid, boolean sync, GlobalTransaction gtx, boolean fromCluster) { if (swallowRemoveRecoveryInfoCalls){ log.trace("PostCommitRecoveryStateTest$RecoveryManagerDelegate.removeRecoveryInformation"); } else { - this.rm.removeRecoveryInformationFromCluster(where, xid, sync, null); + this.rm.removeRecoveryInformation(where, xid, sync, null, false); } } diff --git a/extended-statistics/src/test/java/org/infinispan/stats/BaseTxClusterExtendedStatisticLogicTest.java b/extended-statistics/src/test/java/org/infinispan/stats/BaseTxClusterExtendedStatisticLogicTest.java index d3a68712faa..4181d2dcc23 100644 --- a/extended-statistics/src/test/java/org/infinispan/stats/BaseTxClusterExtendedStatisticLogicTest.java +++ b/extended-statistics/src/test/java/org/infinispan/stats/BaseTxClusterExtendedStatisticLogicTest.java @@ -74,7 +74,7 @@ public long timeDuration(long startTime, long endTime, TimeUnit outputTimeUnit) private final boolean sync2ndPhase; private final boolean totalOrder; private final CacheMode cacheMode; - private final List keys = new ArrayList(128); + private final List keys = new ArrayList<>(128); protected BaseTxClusterExtendedStatisticLogicTest(CacheMode cacheMode, boolean sync2ndPhase, boolean totalOrder) { this.sync = cacheMode.isSynchronous(); @@ -201,9 +201,9 @@ protected void createCacheManagers() throws Throwable { for (int i = 0; i < NUM_NODES; ++i) { lockManagers[i] = extractLockManager(cache(i)); ExtendedStatisticInterceptor interceptor = extendedStatisticInterceptors[i]; - CacheStatisticManager manager = (CacheStatisticManager) extractField(interceptor, "cacheStatisticManager"); - CacheStatisticCollector collector = (CacheStatisticCollector) extractField(manager, "cacheStatisticCollector"); - ConcurrentGlobalContainer globalContainer = (ConcurrentGlobalContainer) extractField(collector, "globalContainer"); + CacheStatisticManager manager = extractField(interceptor, "cacheStatisticManager"); + CacheStatisticCollector collector = extractField(manager, "cacheStatisticCollector"); + ConcurrentGlobalContainer globalContainer = extractField(collector, "globalContainer"); replaceField(TEST_TIME_SERVICE, "timeService", manager, CacheStatisticManager.class); replaceField(TEST_TIME_SERVICE, "timeService", collector, CacheStatisticCollector.class); replaceField(TEST_TIME_SERVICE, "timeService", globalContainer, ConcurrentGlobalContainer.class); @@ -287,10 +287,6 @@ private void testStats(WriteOperation operation, int numOfWriteTx, int numOfWrit if (isRemote(key, cache(txExecutor))) { remotePuts++; involvesRemoteNode = true; - //remote puts always acquires local locks - if (operation == WriteOperation.REPLACE) { - localLocks++; - } } else { localPuts++; }