From c396b4e7d682c45aa2ce70bbe1aa22a30a668a63 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Thu, 16 Aug 2018 19:09:01 +0300 Subject: [PATCH 01/18] IGNITE-9270 WIP --- .../internal/pagemem/wal/WALPointer.java | 2 +- .../processors/cache/CacheEntryExecutor.java | 75 +++ .../processors/cache/CacheEntryOperation.java | 8 + .../cache/CacheEntryOperationCallback.java | 8 + .../cache/CacheEntryOperationFuture.java | 9 + .../processors/cache/CacheEntryRefresh.java | 8 + .../processors/cache/GridCacheProcessor.java | 28 + .../cache/persistence/wal/FileWALPointer.java | 6 +- .../transactions/IgniteTxLocalAdapter.java | 488 +++++++++--------- 9 files changed, 384 insertions(+), 248 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryExecutor.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryOperation.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryOperationCallback.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryOperationFuture.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryRefresh.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/WALPointer.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/WALPointer.java index 7ca4aee09b275..1c8bd7a6432b7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/WALPointer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/WALPointer.java @@ -22,7 +22,7 @@ /** * */ -public interface WALPointer extends Serializable { +public interface WALPointer extends Serializable, Comparable { /** * Pointer to the next record. Can be used only for original pointers obtained from WAL manager. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryExecutor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryExecutor.java new file mode 100644 index 0000000000000..fdbc7a5896169 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryExecutor.java @@ -0,0 +1,75 @@ +package org.apache.ignite.internal.processors.cache; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.util.StripedExecutor; +import org.jetbrains.annotations.Nullable; + +public class CacheEntryExecutor { + + private final StripedExecutor executor; + + public CacheEntryExecutor(StripedExecutor executor) { + this.executor = executor; + } + + public void stop(boolean cancel) { + executor.stop(); + } + + public CacheEntryOperationFuture execute( + GridCacheEntryEx entry, + CacheEntryOperation operationClojure, + @Nullable CacheEntryRefresh refreshClojure, + @Nullable CacheEntryOperationCallback callbackClojure + ) { + assert entry.key() != null : "Entry key is null [entry=" + entry + "]"; + assert entry.key().partition() != -1 : "Entry partition is undefined [entry=" + entry + "]"; + + CacheEntryOperationFuture future = new CacheEntryOperationFuture<>(); + + executor.execute(entry.key().partition(), () -> { + GridCacheEntryEx entry0 = entry; + + R result; + + for (;;) { + try { + result = operationClojure.invoke(entry0); + + if (callbackClojure != null) + callbackClojure.invoke(entry0, result); + + future.onDone(result); + + break; + } + catch (GridCacheEntryRemovedException re) { + if (refreshClojure != null) { + try { + entry0 = refreshClojure.refresh(entry0); + + assert entry0 != null; + } + catch (IgniteCheckedException e) { + future.onDone(e); + + break; + } + } + else { + future.onDone(re); + + break; + } + } + catch (IgniteCheckedException e) { + future.onDone(e); + + break; + } + } + }); + + return future; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryOperation.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryOperation.java new file mode 100644 index 0000000000000..66f53e58f1163 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryOperation.java @@ -0,0 +1,8 @@ +package org.apache.ignite.internal.processors.cache; + +import org.apache.ignite.IgniteCheckedException; + +@FunctionalInterface +public interface CacheEntryOperation { + public R invoke(GridCacheEntryEx entry) throws IgniteCheckedException, GridCacheEntryRemovedException; +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryOperationCallback.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryOperationCallback.java new file mode 100644 index 0000000000000..a7f0aa6e1f0cf --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryOperationCallback.java @@ -0,0 +1,8 @@ +package org.apache.ignite.internal.processors.cache; + +import org.apache.ignite.IgniteCheckedException; + +@FunctionalInterface +public interface CacheEntryOperationCallback { + void invoke(GridCacheEntryEx entry, R result) throws IgniteCheckedException; +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryOperationFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryOperationFuture.java new file mode 100644 index 0000000000000..6102c1e296122 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryOperationFuture.java @@ -0,0 +1,9 @@ +package org.apache.ignite.internal.processors.cache; + +import org.apache.ignite.internal.util.future.GridFutureAdapter; + +public class CacheEntryOperationFuture extends GridFutureAdapter { + + + +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryRefresh.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryRefresh.java new file mode 100644 index 0000000000000..3edddc75c6292 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryRefresh.java @@ -0,0 +1,8 @@ +package org.apache.ignite.internal.processors.cache; + +import org.apache.ignite.IgniteCheckedException; + +@FunctionalInterface +public interface CacheEntryRefresh { + public GridCacheEntryEx refresh(GridCacheEntryEx entry) throws IgniteCheckedException; +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index 6818e50f74559..6abb713cc8fbc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -59,6 +59,7 @@ import org.apache.ignite.configuration.TransactionConfiguration; import org.apache.ignite.configuration.WALMode; import org.apache.ignite.events.EventType; +import org.apache.ignite.failure.FailureContext; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException; import org.apache.ignite.internal.IgniteComponentType; @@ -128,6 +129,7 @@ import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.suggestions.GridPerformanceSuggestions; import org.apache.ignite.internal.util.F0; +import org.apache.ignite.internal.util.StripedExecutor; import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -144,6 +146,7 @@ import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.lifecycle.LifecycleAware; @@ -175,6 +178,7 @@ import static org.apache.ignite.configuration.DeploymentMode.ISOLATED; import static org.apache.ignite.configuration.DeploymentMode.PRIVATE; import static org.apache.ignite.configuration.DeploymentMode.SHARED; +import static org.apache.ignite.failure.FailureType.SYSTEM_WORKER_TERMINATION; import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.CACHE_PROC; import static org.apache.ignite.internal.IgniteComponentType.JTA; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_CONSISTENCY_CHECK_SKIPPED; @@ -205,6 +209,8 @@ public class GridCacheProcessor extends GridProcessorAdapter { /** Shared cache context. */ private GridCacheSharedContext sharedCtx; + private CacheEntryExecutor executor; + /** */ private final ConcurrentMap cacheGrps = new ConcurrentHashMap<>(); @@ -701,6 +707,21 @@ private void cleanup(CacheConfiguration cfg, @Nullable Object rsrc, boolean near ctx.state().cacheProcessorStarted(); ctx.authentication().cacheProcessorStarted(); + + if (!ctx.clientNode() && !ctx.isDaemon()) { + executor = new CacheEntryExecutor(new StripedExecutor( + ctx.config().getStripedPoolSize(), + ctx.config().getIgniteInstanceName(), + "partition-worker", + log, + new IgniteInClosure() { + @Override public void apply(Throwable t) { + ctx.failure().process(new FailureContext(SYSTEM_WORKER_TERMINATION, t)); + } + }, + ctx.workersRegistry() + )); + } } /** @@ -921,6 +942,9 @@ private void checkConsistency() throws IgniteCheckedException { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public void stop(boolean cancel) throws IgniteCheckedException { + if (executor != null) + executor.stop(cancel); + stopCaches(cancel); List> mgrs = sharedCtx.managers(); @@ -4194,6 +4218,10 @@ public GridCacheSharedContext context() { return (GridCacheSharedContext)sharedCtx; } + public CacheEntryExecutor executor() { + return executor; + } + /** * @return Transactions interface implementation. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWALPointer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWALPointer.java index 6ea7e002b6e6d..f93a469863553 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWALPointer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWALPointer.java @@ -24,7 +24,7 @@ /** * File WAL pointer. */ -public class FileWALPointer implements WALPointer, Comparable { +public class FileWALPointer implements WALPointer { /** Serial version uid. */ private static final long serialVersionUID = 0L; @@ -112,7 +112,9 @@ public void length(int len) { } /** {@inheritDoc} */ - @Override public int compareTo(@NotNull FileWALPointer o) { + @Override public int compareTo(@NotNull WALPointer oo) { + FileWALPointer o = (FileWALPointer) oo; + int res = Long.compare(idx, o.idx); return res == 0 ? Integer.compare(fileOff, o.fileOff) : res; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java index 6f11a57500ec2..a7fdc12ea393b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java @@ -18,8 +18,10 @@ package org.apache.ignite.internal.processors.cache.transactions; import java.io.Externalizable; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.UUID; @@ -30,14 +32,16 @@ import javax.cache.processor.EntryProcessor; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cache.CacheWriteSynchronizationMode; -import org.apache.ignite.internal.InvalidEnvironmentException; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.InvalidEnvironmentException; import org.apache.ignite.internal.pagemem.wal.StorageException; import org.apache.ignite.internal.pagemem.wal.WALPointer; import org.apache.ignite.internal.pagemem.wal.record.DataEntry; import org.apache.ignite.internal.pagemem.wal.record.DataRecord; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheEntryOperationFuture; import org.apache.ignite.internal.processors.cache.CacheEntryPredicate; +import org.apache.ignite.internal.processors.cache.CacheEntryRefresh; import org.apache.ignite.internal.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.CacheInvokeEntry; import org.apache.ignite.internal.processors.cache.CacheObject; @@ -62,6 +66,7 @@ import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException; import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException; import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException; +import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.lang.GridClosureException; import org.apache.ignite.internal.util.lang.GridTuple; @@ -510,322 +515,315 @@ protected GridCacheEntryEx entryEx(GridCacheContext cacheCtx, IgniteTxKey key, A AffinityTopologyVersion topVer = topologyVersion(); + List entryCommitFutures = new ArrayList<>(commitEntries.size()); + + GridCompoundFuture commitAllFuture = new GridCompoundFuture(); + /* * Commit to cache. Note that for 'near' transaction we loop through all the entries. */ - for (IgniteTxEntry txEntry : commitEntries) { + for (final IgniteTxEntry txEntry : commitEntries) { GridCacheContext cacheCtx = txEntry.context(); GridDrType drType = cacheCtx.isDrEnabled() ? DR_PRIMARY : DR_NONE; UUID nodeId = txEntry.nodeId() == null ? this.nodeId : txEntry.nodeId(); - try { - while (true) { - try { - GridCacheEntryEx cached = txEntry.cached(); + CacheEntryRefresh refresh = entry -> entryEx(cacheCtx, txEntry.txKey(), topologyVersion()); - // Must try to evict near entries before committing from - // transaction manager to make sure locks are held. - if (!evictNearEntry(txEntry, false)) { - if (cacheCtx.isNear() && cacheCtx.dr().receiveEnabled()) { - cached.markObsolete(xidVer); + try { + GridCacheEntryEx cached = txEntry.cached(); - break; - } + // Must try to evict near entries before committing from + // transaction manager to make sure locks are held. + if (evictNearEntry(txEntry, false)) { + entryCommitFutures.add(new GridFinishedFuture()); - if (cached.detached()) - break; + continue; + } - boolean updateNearCache = updateNearCache(cacheCtx, txEntry.key(), topVer); + if (cacheCtx.isNear() && cacheCtx.dr().receiveEnabled()) { + cached.markObsolete(xidVer); - boolean metrics = true; + continue; + } - if (!updateNearCache && cacheCtx.isNear() && txEntry.locallyMapped()) - metrics = false; + if (cached.detached()) + continue; - boolean evt = !isNearLocallyMapped(txEntry, false); + final boolean updateNearCache = updateNearCache(cacheCtx, txEntry.key(), topVer); - if (!F.isEmpty(txEntry.entryProcessors()) || !F.isEmpty(txEntry.filters())) - txEntry.cached().unswap(false); + final boolean updateMetrics = updateNearCache || !cacheCtx.isNear() || !txEntry.locallyMapped(); - IgniteBiTuple res = applyTransformClosures(txEntry, - true, null); + final boolean fireEvent = !isNearLocallyMapped(txEntry, false); - GridCacheVersion dhtVer = null; + if (!F.isEmpty(txEntry.entryProcessors()) || !F.isEmpty(txEntry.filters())) + txEntry.cached().unswap(false); - // For near local transactions we must record DHT version - // in order to keep near entries on backup nodes until - // backup remote transaction completes. - if (cacheCtx.isNear()) { - if (txEntry.op() == CREATE || txEntry.op() == UPDATE || - txEntry.op() == DELETE || txEntry.op() == TRANSFORM) - dhtVer = txEntry.dhtVersion(); + IgniteBiTuple res = applyTransformClosures(txEntry, + true, null); - if ((txEntry.op() == CREATE || txEntry.op() == UPDATE) && - txEntry.conflictExpireTime() == CU.EXPIRE_TIME_CALCULATE) { - ExpiryPolicy expiry = cacheCtx.expiryForTxEntry(txEntry); + GridCacheVersion dhtVer = null; - if (expiry != null) { - txEntry.cached().unswap(false); + // For near local transactions we must record DHT version + // in order to keep near entries on backup nodes until + // backup remote transaction completes. + if (cacheCtx.isNear()) { + if (txEntry.op() == CREATE || txEntry.op() == UPDATE || + txEntry.op() == DELETE || txEntry.op() == TRANSFORM) + dhtVer = txEntry.dhtVersion(); - Duration duration = cached.hasValue() ? - expiry.getExpiryForUpdate() : expiry.getExpiryForCreation(); + if ((txEntry.op() == CREATE || txEntry.op() == UPDATE) && + txEntry.conflictExpireTime() == CU.EXPIRE_TIME_CALCULATE) { + ExpiryPolicy expiry = cacheCtx.expiryForTxEntry(txEntry); - txEntry.ttl(CU.toTtl(duration)); - } - } - } + if (expiry != null) { + txEntry.cached().unswap(false); - GridCacheOperation op = res.get1(); - CacheObject val = res.get2(); + Duration duration = cached.hasValue() ? + expiry.getExpiryForUpdate() : expiry.getExpiryForCreation(); - // Deal with conflicts. - GridCacheVersion explicitVer = txEntry.conflictVersion() != null ? - txEntry.conflictVersion() : writeVersion(); + txEntry.ttl(CU.toTtl(duration)); + } + } + } - if ((op == CREATE || op == UPDATE) && - txEntry.conflictExpireTime() == CU.EXPIRE_TIME_CALCULATE) { - ExpiryPolicy expiry = cacheCtx.expiryForTxEntry(txEntry); + GridCacheOperation op = res.get1(); + CacheObject val = res.get2(); - if (expiry != null) { - Duration duration = cached.hasValue() ? - expiry.getExpiryForUpdate() : expiry.getExpiryForCreation(); + // Deal with conflicts. + GridCacheVersion explicitVer = txEntry.conflictVersion() != null ? + txEntry.conflictVersion() : writeVersion(); - long ttl = CU.toTtl(duration); + if ((op == CREATE || op == UPDATE) && + txEntry.conflictExpireTime() == CU.EXPIRE_TIME_CALCULATE) { + ExpiryPolicy expiry = cacheCtx.expiryForTxEntry(txEntry); - txEntry.ttl(ttl); + if (expiry != null) { + Duration duration = cached.hasValue() ? + expiry.getExpiryForUpdate() : expiry.getExpiryForCreation(); - if (ttl == CU.TTL_ZERO) - op = DELETE; - } - } + long ttl = CU.toTtl(duration); - boolean conflictNeedResolve = cacheCtx.conflictNeedResolve(); + txEntry.ttl(ttl); - GridCacheVersionConflictContext conflictCtx = null; + if (ttl == CU.TTL_ZERO) + op = DELETE; + } + } - if (conflictNeedResolve) { - IgniteBiTuple conflictRes = - conflictResolve(op, txEntry, val, explicitVer, cached); + boolean conflictNeedResolve = cacheCtx.conflictNeedResolve(); - assert conflictRes != null; + GridCacheVersionConflictContext conflictCtx = null; - conflictCtx = conflictRes.get2(); + if (conflictNeedResolve) { + IgniteBiTuple conflictRes = + conflictResolve(op, txEntry, val, explicitVer, cached); - if (conflictCtx.isUseOld()) - op = NOOP; - else if (conflictCtx.isUseNew()) { - txEntry.ttl(conflictCtx.ttl()); - txEntry.conflictExpireTime(conflictCtx.expireTime()); - } - else { - assert conflictCtx.isMerge(); + assert conflictRes != null; - op = conflictRes.get1(); - val = txEntry.context().toCacheObject(conflictCtx.mergeValue()); - explicitVer = writeVersion(); + conflictCtx = conflictRes.get2(); - txEntry.ttl(conflictCtx.ttl()); - txEntry.conflictExpireTime(conflictCtx.expireTime()); - } - } - else - // Nullify explicit version so that innerSet/innerRemove will work as usual. - explicitVer = null; + if (conflictCtx.isUseOld()) + op = NOOP; + else if (conflictCtx.isUseNew()) { + txEntry.ttl(conflictCtx.ttl()); + txEntry.conflictExpireTime(conflictCtx.expireTime()); + } + else { + assert conflictCtx.isMerge(); - if (sndTransformedVals || conflictNeedResolve) { - assert sndTransformedVals && cacheCtx.isReplicated() || conflictNeedResolve; + op = conflictRes.get1(); + val = txEntry.context().toCacheObject(conflictCtx.mergeValue()); + explicitVer = writeVersion(); - txEntry.value(val, true, false); - txEntry.op(op); - txEntry.entryProcessors(null); - txEntry.conflictVersion(explicitVer); - } + txEntry.ttl(conflictCtx.ttl()); + txEntry.conflictExpireTime(conflictCtx.expireTime()); + } + } + else + // Nullify explicit version so that innerSet/innerRemove will work as usual. + explicitVer = null; - if (dhtVer == null) - dhtVer = explicitVer != null ? explicitVer : writeVersion(); + if (sndTransformedVals || conflictNeedResolve) { + assert sndTransformedVals && cacheCtx.isReplicated() || conflictNeedResolve; - if (op == CREATE || op == UPDATE) { - assert val != null : txEntry; + txEntry.value(val, true, false); + txEntry.op(op); + txEntry.entryProcessors(null); + txEntry.conflictVersion(explicitVer); + } - GridCacheUpdateTxResult updRes = cached.innerSet( - this, + if (dhtVer == null) + dhtVer = explicitVer != null ? explicitVer : writeVersion(); + + final GridCacheVersion explicitVer0 = explicitVer; + final GridCacheVersion dhtVer0 = dhtVer; + final CacheObject val0 = val; + + if (op == CREATE || op == UPDATE) { + assert val != null : txEntry; + + CacheEntryOperationFuture fut = cctx.cache().executor().execute(cached, e -> e.innerSet( + this, + eventNodeId(), + txEntry.nodeId(), + val0, + false, + false, + txEntry.ttl(), + fireEvent, + updateMetrics, + txEntry.keepBinary(), + txEntry.hasOldValue(), + txEntry.oldValue(), + topVer, + null, + cached.detached() ? DR_NONE : drType, + txEntry.conflictExpireTime(), + cached.isNear() ? null : explicitVer0, + CU.subjectId(this, cctx), + resolveTaskName(), + dhtVer0, + null), + refresh, + ((entry, result) -> { + if (updateNearCache && result.success()) { + updateNearEntrySafely(cacheCtx, txEntry.key(), nearEntry -> nearEntry.innerSet( + null, eventNodeId(), - txEntry.nodeId(), - val, + nodeId, + val0, false, false, txEntry.ttl(), - evt, - metrics, + false, + updateMetrics, txEntry.keepBinary(), txEntry.hasOldValue(), txEntry.oldValue(), topVer, - null, - cached.detached() ? DR_NONE : drType, + CU.empty0(), + DR_NONE, txEntry.conflictExpireTime(), - cached.isNear() ? null : explicitVer, + null, CU.subjectId(this, cctx), resolveTaskName(), - dhtVer, - null); - - if (updRes.success()) - txEntry.updateCounter(updRes.updatePartitionCounter()); - - if (updRes.loggedPointer() != null) - ptr = updRes.loggedPointer(); - - if (updRes.success() && updateNearCache) { - final CacheObject val0 = val; - final boolean metrics0 = metrics; - final GridCacheVersion dhtVer0 = dhtVer; - - updateNearEntrySafely(cacheCtx, txEntry.key(), entry -> entry.innerSet( - null, - eventNodeId(), - nodeId, - val0, - false, - false, - txEntry.ttl(), - false, - metrics0, - txEntry.keepBinary(), - txEntry.hasOldValue(), - txEntry.oldValue(), - topVer, - CU.empty0(), - DR_NONE, - txEntry.conflictExpireTime(), - null, - CU.subjectId(this, cctx), - resolveTaskName(), - dhtVer0, - null) - ); - } + dhtVer0, + null) + ); } - else if (op == DELETE) { - GridCacheUpdateTxResult updRes = cached.innerRemove( - this, + }) + ); + + entryCommitFutures.add(fut); + } + else if (op == DELETE) { + CacheEntryOperationFuture fut = cctx.cache().executor().execute(cached, e -> e.innerRemove( + this, + eventNodeId(), + txEntry.nodeId(), + false, + fireEvent, + updateMetrics, + txEntry.keepBinary(), + txEntry.hasOldValue(), + txEntry.oldValue(), + topVer, + null, + cached.detached() ? DR_NONE : drType, + cached.isNear() ? null : explicitVer0, + CU.subjectId(this, cctx), + resolveTaskName(), + dhtVer0, + null), + refresh, + ((entry, result) -> { + if (updateNearCache && result.success()) { + updateNearEntrySafely(cacheCtx, txEntry.key(), nearEntry -> nearEntry.innerRemove( + null, eventNodeId(), - txEntry.nodeId(), + nodeId, + false, false, - evt, - metrics, + updateMetrics, txEntry.keepBinary(), txEntry.hasOldValue(), txEntry.oldValue(), topVer, + CU.empty0(), + DR_NONE, null, - cached.detached() ? DR_NONE : drType, - cached.isNear() ? null : explicitVer, CU.subjectId(this, cctx), resolveTaskName(), - dhtVer, - null); - - if (updRes.success()) - txEntry.updateCounter(updRes.updatePartitionCounter()); - - if (updRes.loggedPointer() != null) - ptr = updRes.loggedPointer(); - - if (updRes.success() && updateNearCache) { - final boolean metrics0 = metrics; - final GridCacheVersion dhtVer0 = dhtVer; - - updateNearEntrySafely(cacheCtx, txEntry.key(), entry -> entry.innerRemove( - null, - eventNodeId(), - nodeId, - false, - false, - metrics0, - txEntry.keepBinary(), - txEntry.hasOldValue(), - txEntry.oldValue(), - topVer, - CU.empty0(), - DR_NONE, - null, - CU.subjectId(this, cctx), - resolveTaskName(), - dhtVer0, - null) - ); - } + dhtVer0, + null) + ); } - else if (op == RELOAD) { - cached.innerReload(); - if (updateNearCache) - updateNearEntrySafely(cacheCtx, txEntry.key(), entry -> entry.innerReload()); - } - else if (op == READ) { - CacheGroupContext grp = cacheCtx.group(); - - if (grp.persistenceEnabled() && grp.walEnabled() && - cctx.snapshot().needTxReadLogging()) { - ptr = cctx.wal().log(new DataRecord(new DataEntry( - cacheCtx.cacheId(), - txEntry.key(), - val, - op, - nearXidVersion(), - writeVersion(), - 0, - txEntry.key().partition(), - txEntry.updateCounter()))); - } - - ExpiryPolicy expiry = cacheCtx.expiryForTxEntry(txEntry); - - if (expiry != null) { - Duration duration = expiry.getExpiryForAccess(); - - if (duration != null) - cached.updateTtl(null, CU.toTtl(duration)); - } - - if (log.isDebugEnabled()) - log.debug("Ignoring READ entry when committing: " + txEntry); - } - else { - assert ownsLock(txEntry.cached()): - "Transaction does not own lock for group lock entry during commit [tx=" + - this + ", txEntry=" + txEntry + ']'; - - if (conflictCtx == null || !conflictCtx.isUseOld()) { - if (txEntry.ttl() != CU.TTL_NOT_CHANGED) - cached.updateTtl(null, txEntry.ttl()); - } - - if (log.isDebugEnabled()) - log.debug("Ignoring NOOP entry when committing: " + txEntry); - } - } + // Check commit locks after set, to make sure that + // we are not changing obsolete entries. + // (innerSet and innerRemove will throw an exception + // if an entry is obsolete). + if (txEntry.op() != READ) + checkCommitLocks(entry); + }) + ); - // Check commit locks after set, to make sure that - // we are not changing obsolete entries. - // (innerSet and innerRemove will throw an exception - // if an entry is obsolete). - if (txEntry.op() != READ) - checkCommitLocks(cached); + entryCommitFutures.add(fut); + } + else if (op == RELOAD) { + CacheEntryOperationFuture fut = cctx.cache().executor().execute( + cached, e -> e.innerReload(), + refresh, + (entry, result) -> { + if (updateNearCache) + updateNearEntrySafely(cacheCtx, txEntry.key(), nearEntry -> nearEntry.innerReload()); + }); + + entryCommitFutures.add(fut); + } + else if (op == READ) { + CacheGroupContext grp = cacheCtx.group(); + + if (grp.persistenceEnabled() && grp.walEnabled() && + cctx.snapshot().needTxReadLogging()) { + ptr = cctx.wal().log(new DataRecord(new DataEntry( + cacheCtx.cacheId(), + txEntry.key(), + val, + op, + nearXidVersion(), + writeVersion(), + 0, + txEntry.key().partition(), + txEntry.updateCounter()))); + } - // Break out of while loop. - break; + ExpiryPolicy expiry = cacheCtx.expiryForTxEntry(txEntry); + + if (expiry != null) { + Duration duration = expiry.getExpiryForAccess(); + + if (duration != null) + cached.updateTtl(null, CU.toTtl(duration)); } - // If entry cached within transaction got removed. - catch (GridCacheEntryRemovedException ignored) { - if (log.isDebugEnabled()) - log.debug("Got removed entry during transaction commit (will retry): " + txEntry); - txEntry.cached(entryEx(cacheCtx, txEntry.txKey(), topologyVersion())); + if (log.isDebugEnabled()) + log.debug("Ignoring READ entry when committing: " + txEntry); + } + else { + assert ownsLock(txEntry.cached()): + "Transaction does not own lock for group lock entry during commit [tx=" + + this + ", txEntry=" + txEntry + ']'; + + if (conflictCtx == null || !conflictCtx.isUseOld()) { + if (txEntry.ttl() != CU.TTL_NOT_CHANGED) + cached.updateTtl(null, txEntry.ttl()); } + + if (log.isDebugEnabled()) + log.debug("Ignoring NOOP entry when committing: " + txEntry); } } catch (Throwable ex) { From 4c8ebbd5e8176c93583e1d6dd808608a0fa69c3d Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Thu, 16 Aug 2018 20:23:43 +0300 Subject: [PATCH 02/18] IGNITE-9270 WIP --- .../processors/cache/CacheEntryExecutor.java | 6 + .../cache/GridCacheUpdateTxResult.java | 4 +- .../transactions/IgniteTxLocalAdapter.java | 231 +++++++++++------- .../util/future/GridCompoundFuture.java | 13 +- 4 files changed, 159 insertions(+), 95 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryExecutor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryExecutor.java index fdbc7a5896169..2a8b84036226a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryExecutor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryExecutor.java @@ -72,4 +72,10 @@ public CacheEntryOperationFuture execute( return future; } + + static class ExecutionContext { + + private GridCacheEntryEx entry; + + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUpdateTxResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUpdateTxResult.java index 92af83b11fa91..b0650fd2e6b6c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUpdateTxResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUpdateTxResult.java @@ -34,7 +34,7 @@ public class GridCacheUpdateTxResult { private final CacheObject oldVal; /** Partition idx. */ - private long updateCntr; + private long updateCntr = -1; /** */ private WALPointer logPtr; @@ -46,7 +46,7 @@ public class GridCacheUpdateTxResult { * @param oldVal Old value (if any), * @param logPtr Logger WAL pointer for the update. */ - GridCacheUpdateTxResult(boolean success, @Nullable CacheObject oldVal, WALPointer logPtr) { + public GridCacheUpdateTxResult(boolean success, @Nullable CacheObject oldVal, WALPointer logPtr) { this.success = success; this.oldVal = oldVal; this.logPtr = logPtr; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java index a7fdc12ea393b..6feeed4f82832 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java @@ -515,9 +515,7 @@ protected GridCacheEntryEx entryEx(GridCacheContext cacheCtx, IgniteTxKey key, A AffinityTopologyVersion topVer = topologyVersion(); - List entryCommitFutures = new ArrayList<>(commitEntries.size()); - - GridCompoundFuture commitAllFuture = new GridCompoundFuture(); + GridCompoundFuture entriesCommitFuture = new GridCompoundFuture<>(); /* * Commit to cache. Note that for 'near' transaction we loop through all the entries. @@ -537,7 +535,7 @@ protected GridCacheEntryEx entryEx(GridCacheContext cacheCtx, IgniteTxKey key, A // Must try to evict near entries before committing from // transaction manager to make sure locks are held. if (evictNearEntry(txEntry, false)) { - entryCommitFutures.add(new GridFinishedFuture()); + entriesCommitFuture.add(new GridFinishedFuture<>()); continue; } @@ -545,11 +543,17 @@ protected GridCacheEntryEx entryEx(GridCacheContext cacheCtx, IgniteTxKey key, A if (cacheCtx.isNear() && cacheCtx.dr().receiveEnabled()) { cached.markObsolete(xidVer); + entriesCommitFuture.add(new GridFinishedFuture<>()); + continue; } - if (cached.detached()) + if (cached.detached()) { + + entriesCommitFuture.add(new GridFinishedFuture<>()); + continue; + } final boolean updateNearCache = updateNearCache(cacheCtx, txEntry.key(), topVer); @@ -661,10 +665,12 @@ else if (conflictCtx.isUseNew()) { final GridCacheVersion dhtVer0 = dhtVer; final CacheObject val0 = val; + CacheEntryOperationFuture operationFuture; + if (op == CREATE || op == UPDATE) { assert val != null : txEntry; - CacheEntryOperationFuture fut = cctx.cache().executor().execute(cached, e -> e.innerSet( + operationFuture = cctx.cache().executor().execute(cached, e -> e.innerSet( this, eventNodeId(), txEntry.nodeId(), @@ -715,11 +721,9 @@ else if (conflictCtx.isUseNew()) { } }) ); - - entryCommitFutures.add(fut); } else if (op == DELETE) { - CacheEntryOperationFuture fut = cctx.cache().executor().execute(cached, e -> e.innerRemove( + operationFuture = cctx.cache().executor().execute(cached, e -> e.innerRemove( this, eventNodeId(), txEntry.nodeId(), @@ -769,115 +773,170 @@ else if (op == DELETE) { checkCommitLocks(entry); }) ); - - entryCommitFutures.add(fut); } else if (op == RELOAD) { - CacheEntryOperationFuture fut = cctx.cache().executor().execute( - cached, e -> e.innerReload(), + operationFuture = cctx.cache().executor().execute( + cached, e -> { + e.innerReload(); + + return new GridCacheUpdateTxResult( + true, + null, + null + ); + }, refresh, (entry, result) -> { - if (updateNearCache) + if (updateNearCache && result.success()) updateNearEntrySafely(cacheCtx, txEntry.key(), nearEntry -> nearEntry.innerReload()); - }); - - entryCommitFutures.add(fut); + } + ); } else if (op == READ) { - CacheGroupContext grp = cacheCtx.group(); - - if (grp.persistenceEnabled() && grp.walEnabled() && - cctx.snapshot().needTxReadLogging()) { - ptr = cctx.wal().log(new DataRecord(new DataEntry( - cacheCtx.cacheId(), - txEntry.key(), - val, - op, - nearXidVersion(), - writeVersion(), - 0, - txEntry.key().partition(), - txEntry.updateCounter()))); - } + operationFuture = cctx.cache().executor().execute( + cached, e -> { + CacheGroupContext grp = cacheCtx.group(); + WALPointer ptr0 = null; + + if (grp.persistenceEnabled() && grp.walEnabled() && + cctx.snapshot().needTxReadLogging()) { + ptr0 = cctx.wal().log(new DataRecord(new DataEntry( + cacheCtx.cacheId(), + txEntry.key(), + val0, + READ, + nearXidVersion(), + writeVersion(), + 0, + txEntry.key().partition(), + txEntry.updateCounter()))); + } - ExpiryPolicy expiry = cacheCtx.expiryForTxEntry(txEntry); + ExpiryPolicy expiry = cacheCtx.expiryForTxEntry(txEntry); - if (expiry != null) { - Duration duration = expiry.getExpiryForAccess(); + if (expiry != null) { + Duration duration = expiry.getExpiryForAccess(); - if (duration != null) - cached.updateTtl(null, CU.toTtl(duration)); - } + if (duration != null) + cached.updateTtl(null, CU.toTtl(duration)); + } - if (log.isDebugEnabled()) - log.debug("Ignoring READ entry when committing: " + txEntry); + if (log.isDebugEnabled()) + log.debug("Ignoring READ entry when committing: " + txEntry); + + return new GridCacheUpdateTxResult( + true, + val0, + ptr0 + ); + }, + refresh, + (entry, result) -> { } + ); } else { - assert ownsLock(txEntry.cached()): - "Transaction does not own lock for group lock entry during commit [tx=" + - this + ", txEntry=" + txEntry + ']'; + GridCacheVersionConflictContext conflictCtx0 = conflictCtx; - if (conflictCtx == null || !conflictCtx.isUseOld()) { - if (txEntry.ttl() != CU.TTL_NOT_CHANGED) - cached.updateTtl(null, txEntry.ttl()); - } + operationFuture = cctx.cache().executor().execute( + cached, e -> { + assert ownsLock(txEntry.cached()): + "Transaction does not own lock for group lock entry during commit [tx=" + + this + ", txEntry=" + txEntry + ']'; - if (log.isDebugEnabled()) - log.debug("Ignoring NOOP entry when committing: " + txEntry); + if (conflictCtx0 == null || !conflictCtx0.isUseOld()) { + if (txEntry.ttl() != CU.TTL_NOT_CHANGED) + cached.updateTtl(null, txEntry.ttl()); + } + + if (log.isDebugEnabled()) + log.debug("Ignoring NOOP entry when committing: " + txEntry); + + return new GridCacheUpdateTxResult( + false, + val0, + null + ); + }, + refresh, + (entry, result) -> { } + ); } + + assert operationFuture != null : "Entry operation must be submitted"; + + entriesCommitFuture.add(operationFuture); } catch (Throwable ex) { - // We are about to initiate transaction rollback when tx has started to committing. - // Need to remove version from committed list. - cctx.tm().removeCommittedTx(this); + entriesCommitFuture.onDone(ex); - if (X.hasCause(ex, GridCacheIndexUpdateException.class) && cacheCtx.cache().isMongoDataCache()) { - if (log.isDebugEnabled()) - log.debug("Failed to update mongo document index (transaction entry will " + - "be ignored): " + txEntry); + break; + } + } - // Set operation to NOOP. - txEntry.op(NOOP); + entriesCommitFuture.markInitialized(); - errorWhenCommitting(); + try { + entriesCommitFuture.get(); - throw ex; + assert commitEntries.size() == entriesCommitFuture.futures().size(); + + int txEntryIdx = 0; + + for (IgniteTxEntry txEntry : commitEntries) { + GridCacheUpdateTxResult txResult = entriesCommitFuture.future(txEntryIdx).get(); + + if (txResult != null && txResult.success()) { + if (txResult.updatePartitionCounter() != -1) + txEntry.updateCounter(txResult.updatePartitionCounter()); + + WALPointer loggedPtr = txResult.loggedPointer(); + + // Find latest logged WAL pointer. + if (loggedPtr != null) + if (ptr == null || loggedPtr.compareTo(ptr) > 0) + ptr = loggedPtr; } - else { - boolean hasInvalidEnvironmentIssue = X.hasCause(ex, InvalidEnvironmentException.class); - IgniteCheckedException err = new IgniteTxHeuristicCheckedException("Failed to locally write to cache " + - "(all transaction entries will be invalidated, however there was a window when " + - "entries for this transaction were visible to others): " + this, ex); + txEntryIdx++; + } + } + catch (Throwable ex) { + // We are about to initiate transaction rollback when tx has started to committing. + // Need to remove version from committed list. + cctx.tm().removeCommittedTx(this); - if (hasInvalidEnvironmentIssue) { - U.warn(log, "Failed to commit transaction, node is stopping " + - "[tx=" + this + ", err=" + ex + ']'); - } - else - U.error(log, "Heuristic transaction failure.", err); + boolean hasInvalidEnvironmentIssue = X.hasCause(ex, InvalidEnvironmentException.class); - COMMIT_ERR_UPD.compareAndSet(this, null, err); + IgniteCheckedException err = new IgniteTxHeuristicCheckedException("Failed to locally write to cache " + + "(all transaction entries will be invalidated, however there was a window when " + + "entries for this transaction were visible to others): " + this, ex); - state(UNKNOWN); + if (hasInvalidEnvironmentIssue) { + U.warn(log, "Failed to commit transaction, node is stopping " + + "[tx=" + this + ", err=" + ex + ']'); + } + else + U.error(log, "Heuristic transaction failure.", err); - try { - // Courtesy to minimize damage. - uncommit(hasInvalidEnvironmentIssue); - } - catch (Throwable ex1) { - U.error(log, "Failed to uncommit transaction: " + this, ex1); + COMMIT_ERR_UPD.compareAndSet(this, null, err); - if (ex1 instanceof Error) - throw ex1; - } + state(UNKNOWN); - if (ex instanceof Error) - throw ex; + try { + // Courtesy to minimize damage. + uncommit(hasInvalidEnvironmentIssue); + } + catch (Throwable ex1) { + U.error(log, "Failed to uncommit transaction: " + this, ex1); - throw err; - } + if (ex1 instanceof Error) + throw ex1; } + + if (ex instanceof Error) + throw ex; + + throw err; } if (ptr != null && !cctx.tm().logTxRecords()) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java index a7240601ef0c0..28ce52f9337c5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java @@ -17,6 +17,11 @@ package org.apache.ignite.internal.util.future; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.IgniteFutureCancelledCheckedException; @@ -33,12 +38,6 @@ import org.apache.ignite.lang.IgniteReducer; import org.jetbrains.annotations.Nullable; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; - /** * Future composed of multiple inner futures. */ @@ -324,7 +323,7 @@ protected void logDebug(IgniteLogger log, String msg) { * @return Future. */ @SuppressWarnings("unchecked") - protected final IgniteInternalFuture future(int idx) { + public final IgniteInternalFuture future(int idx) { assert Thread.holdsLock(this); assert futs != null && idx >= 0 && idx < futuresCountNoLock(); From 1778b8e8b71d748f5305cc3f0e4a21c35c7b3067 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Thu, 16 Aug 2018 20:41:21 +0300 Subject: [PATCH 03/18] IGNITE-9270 WIP --- .../cache/transactions/IgniteTxLocalAdapter.java | 7 ++++--- .../ignite/internal/util/future/GridCompoundFuture.java | 4 ++-- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java index 6feeed4f82832..9c381232ff350 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java @@ -549,7 +549,6 @@ protected GridCacheEntryEx entryEx(GridCacheContext cacheCtx, IgniteTxKey key, A } if (cached.detached()) { - entriesCommitFuture.add(new GridFinishedFuture<>()); continue; @@ -878,12 +877,14 @@ assert ownsLock(txEntry.cached()): try { entriesCommitFuture.get(); - assert commitEntries.size() == entriesCommitFuture.futures().size(); + List> futures = entriesCommitFuture.futures(); + + assert commitEntries.size() == futures.size(); int txEntryIdx = 0; for (IgniteTxEntry txEntry : commitEntries) { - GridCacheUpdateTxResult txResult = entriesCommitFuture.future(txEntryIdx).get(); + GridCacheUpdateTxResult txResult = futures.get(txEntryIdx).get(); if (txResult != null && txResult.success()) { if (txResult.updatePartitionCounter() != -1) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java index 28ce52f9337c5..0fdf0c28058e5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java @@ -161,7 +161,7 @@ public GridCompoundFuture(@Nullable IgniteReducer rdc) { * @return Collection of futures. */ @SuppressWarnings("unchecked") - public final synchronized Collection> futures() { + public final synchronized List> futures() { if (futs == null) return Collections.emptyList(); @@ -323,7 +323,7 @@ protected void logDebug(IgniteLogger log, String msg) { * @return Future. */ @SuppressWarnings("unchecked") - public final IgniteInternalFuture future(int idx) { + protected final IgniteInternalFuture future(int idx) { assert Thread.holdsLock(this); assert futs != null && idx >= 0 && idx < futuresCountNoLock(); From f6c7621704a63b411e71e87fd0d048ff1a80755e Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Tue, 21 Aug 2018 16:10:04 +0300 Subject: [PATCH 04/18] IGNITE-9270 WIP --- .../apache/ignite/internal/IgnitionEx.java | 6 +- .../processors/cache/CacheEntryExecutor.java | 11 +- .../processors/cache/GridCacheProcessor.java | 7 +- .../dht/GridDhtTxFinishFuture.java | 181 ++++++------- .../cache/distributed/dht/GridDhtTxLocal.java | 79 +++--- .../dht/GridDhtTxLocalAdapter.java | 47 +++- .../near/GridNearTxFinishFuture.java | 162 +++++++----- .../distributed/near/GridNearTxLocal.java | 160 ++++-------- .../IgniteTxCommitEntriesFuture.java | 25 ++ .../transactions/IgniteTxCommitFuture.java | 44 ++++ .../cache/transactions/IgniteTxFinisher.java | 116 +++++++++ .../cache/transactions/IgniteTxHandler.java | 8 +- .../transactions/IgniteTxLocalAdapter.java | 244 ++++++++++-------- .../cache/transactions/IgniteTxLocalEx.java | 6 +- .../cache/transactions/IgniteTxManager.java | 8 + .../ignite/internal/util/StripedExecutor.java | 38 ++- .../internal/util/StripedExecutorTest.java | 2 +- 17 files changed, 722 insertions(+), 422 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxCommitEntriesFuture.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxCommitFuture.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java index 2b408df7ef379..75f6896c6d26a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java @@ -1830,7 +1830,8 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { grid.context().failure().process(new FailureContext(SYSTEM_WORKER_TERMINATION, t)); } }, - workerRegistry); + workerRegistry, + 1); // Note that since we use 'LinkedBlockingQueue', number of // maximum threads has no effect. @@ -1879,7 +1880,8 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { } }, true, - workerRegistry); + workerRegistry, + 0); // Note that we do not pre-start threads here as igfs pool may not be needed. validateThreadPoolSize(cfg.getIgfsThreadPoolSize(), "IGFS"); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryExecutor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryExecutor.java index 2a8b84036226a..a460ed891589d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryExecutor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryExecutor.java @@ -8,8 +8,12 @@ public class CacheEntryExecutor { private final StripedExecutor executor; - public CacheEntryExecutor(StripedExecutor executor) { + private final GridCacheSharedContext cctx; + + + public CacheEntryExecutor(StripedExecutor executor, GridCacheSharedContext cctx) { this.executor = executor; + this.cctx = cctx; } public void stop(boolean cancel) { @@ -33,6 +37,8 @@ public CacheEntryOperationFuture execute( R result; for (;;) { + cctx.database().checkpointReadLock(); + try { result = operationClojure.invoke(entry0); @@ -67,6 +73,9 @@ public CacheEntryOperationFuture execute( break; } + finally { + cctx.database().checkpointReadUnlock(); + } } }); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index 6abb713cc8fbc..0118fa8cbc9bd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -708,7 +708,7 @@ private void cleanup(CacheConfiguration cfg, @Nullable Object rsrc, boolean near ctx.state().cacheProcessorStarted(); ctx.authentication().cacheProcessorStarted(); - if (!ctx.clientNode() && !ctx.isDaemon()) { + if (!ctx.isDaemon()) { executor = new CacheEntryExecutor(new StripedExecutor( ctx.config().getStripedPoolSize(), ctx.config().getIgniteInstanceName(), @@ -719,8 +719,9 @@ private void cleanup(CacheConfiguration cfg, @Nullable Object rsrc, boolean near ctx.failure().process(new FailureContext(SYSTEM_WORKER_TERMINATION, t)); } }, - ctx.workersRegistry() - )); + ctx.workersRegistry(), + 0 + ), sharedCtx); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java index 0ed8419b32297..9edf3e6e92743 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java @@ -291,6 +291,8 @@ else if (!commit && !F.isEmpty(tx.lockTransactionNodes())) // No backup or near nodes to send commit message to (just complete then). sync = false; + cctx.tm().finisher().finishSend(tx); + markInitialized(); if (!sync) @@ -304,15 +306,12 @@ else if (!commit && !F.isEmpty(tx.lockTransactionNodes())) private boolean rollbackLockTransactions(Collection nodes) { assert !F.isEmpty(nodes); - if (tx.onePhaseCommit()) - return false; - - boolean sync = tx.syncMode() == FULL_SYNC; + if (tx.onePhaseCommit()) { - if (tx.explicitLock()) - sync = true; + return false; + } - boolean res = false; + final boolean sync = tx.explicitLock() || tx.syncMode() == FULL_SYNC; int miniId = 0; @@ -349,38 +348,40 @@ private boolean rollbackLockTransactions(Collection nodes) { false, false); - try { - cctx.io().send(n, req, tx.ioPolicy()); - - if (msgLog.isDebugEnabled()) { - msgLog.debug("DHT finish fut, sent request lock tx [txId=" + tx.nearXidVersion() + - ", dhtTxId=" + tx.xidVersion() + - ", node=" + n.id() + ']'); - } + cctx.tm().finisher().send(tx, () -> { + try { + cctx.io().send(n, req, tx.ioPolicy()); - if (sync) - res = true; - else - fut.onDone(); - } - catch (IgniteCheckedException e) { - // Fail the whole thing. - if (e instanceof ClusterTopologyCheckedException) - fut.onNodeLeft((ClusterTopologyCheckedException)e); - else { if (msgLog.isDebugEnabled()) { - msgLog.debug("DHT finish fut, failed to send request lock tx [txId=" + tx.nearXidVersion() + + msgLog.debug("DHT finish fut, sent request lock tx [txId=" + tx.nearXidVersion() + ", dhtTxId=" + tx.xidVersion() + - ", node=" + n.id() + - ", err=" + e + ']'); + ", node=" + n.id() + ']'); } - fut.onResult(e); + if (!sync) + fut.onDone(); } - } + catch (IgniteCheckedException e) { + // Fail the whole thing. + if (e instanceof ClusterTopologyCheckedException) + fut.onNodeLeft((ClusterTopologyCheckedException)e); + else { + if (msgLog.isDebugEnabled()) { + msgLog.debug("DHT finish fut, failed to send request lock tx [txId=" + tx.nearXidVersion() + + ", dhtTxId=" + tx.xidVersion() + + ", node=" + n.id() + + ", err=" + e + ']'); + } + + fut.onResult(e); + } + } + }); } - return res; + cctx.tm().finisher().finishSend(tx); + + return sync && miniId > 0; } /** @@ -389,21 +390,21 @@ private boolean rollbackLockTransactions(Collection nodes) { * @param nearMap Near map. * @return {@code True} in case there is at least one synchronous {@code MiniFuture} to wait for. */ - private boolean finish(boolean commit, + private boolean finish( + boolean commit, Map dhtMap, - Map nearMap) { - if (tx.onePhaseCommit()) + Map nearMap + ) { + if (tx.onePhaseCommit()) { return false; + } - boolean sync = tx.syncMode() == FULL_SYNC; - - if (tx.explicitLock()) - sync = true; - - boolean res = false; + final boolean sync = tx.explicitLock() || (tx.syncMode() == FULL_SYNC); int miniId = 0; + boolean hasMini = false; + // Create mini futures. for (GridDistributedTxMapping dhtMapping : dhtMap.values()) { ClusterNode n = dhtMapping.primary(); @@ -454,35 +455,38 @@ private boolean finish(boolean commit, req.writeVersion(tx.writeVersion() != null ? tx.writeVersion() : tx.xidVersion()); - try { - cctx.io().send(n, req, tx.ioPolicy()); - - if (msgLog.isDebugEnabled()) { - msgLog.debug("DHT finish fut, sent request dht [txId=" + tx.nearXidVersion() + - ", dhtTxId=" + tx.xidVersion() + - ", node=" + n.id() + ']'); - } + cctx.tm().finisher().send(tx, () -> { + try { + cctx.io().send(n, req, tx.ioPolicy()); - if (sync) - res = true; - else - fut.onDone(); - } - catch (IgniteCheckedException e) { - // Fail the whole thing. - if (e instanceof ClusterTopologyCheckedException) - fut.onNodeLeft((ClusterTopologyCheckedException)e); - else { if (msgLog.isDebugEnabled()) { - msgLog.debug("DHT finish fut, failed to send request dht [txId=" + tx.nearXidVersion() + + msgLog.debug("DHT finish fut, sent request dht [txId=" + tx.nearXidVersion() + ", dhtTxId=" + tx.xidVersion() + - ", node=" + n.id() + - ", err=" + e + ']'); + ", node=" + n.id() + ']'); } - fut.onResult(e); + if (!sync) + fut.onDone(); } - } + catch (IgniteCheckedException e) { + // Fail the whole thing. + if (e instanceof ClusterTopologyCheckedException) + fut.onNodeLeft((ClusterTopologyCheckedException)e); + else { + if (msgLog.isDebugEnabled()) { + msgLog.debug("DHT finish fut, failed to send request dht [txId=" + tx.nearXidVersion() + + ", dhtTxId=" + tx.xidVersion() + + ", node=" + n.id() + + ", err=" + e + ']'); + } + + fut.onResult(e); + } + } + }); + + if (sync) + hasMini = true; } for (GridDistributedTxMapping nearMapping : nearMap.values()) { @@ -523,39 +527,42 @@ private boolean finish(boolean commit, req.writeVersion(tx.writeVersion()); - try { - cctx.io().send(nearMapping.primary(), req, tx.ioPolicy()); + cctx.tm().finisher().send(tx, () -> { + try { + cctx.io().send(nearMapping.primary(), req, tx.ioPolicy()); - if (msgLog.isDebugEnabled()) { - msgLog.debug("DHT finish fut, sent request near [txId=" + tx.nearXidVersion() + - ", dhtTxId=" + tx.xidVersion() + - ", node=" + nearMapping.primary().id() + ']'); - } - - if (sync) - res = true; - else - fut.onDone(); - } - catch (IgniteCheckedException e) { - // Fail the whole thing. - if (e instanceof ClusterTopologyCheckedException) - fut.onNodeLeft((ClusterTopologyCheckedException)e); - else { if (msgLog.isDebugEnabled()) { - msgLog.debug("DHT finish fut, failed to send request near [txId=" + tx.nearXidVersion() + + msgLog.debug("DHT finish fut, sent request near [txId=" + tx.nearXidVersion() + ", dhtTxId=" + tx.xidVersion() + - ", node=" + nearMapping.primary().id() + - ", err=" + e + ']'); + ", node=" + nearMapping.primary().id() + ']'); } - fut.onResult(e); + if (!sync) + fut.onDone(); } - } + catch (IgniteCheckedException e) { + // Fail the whole thing. + if (e instanceof ClusterTopologyCheckedException) + fut.onNodeLeft((ClusterTopologyCheckedException)e); + else { + if (msgLog.isDebugEnabled()) { + msgLog.debug("DHT finish fut, failed to send request near [txId=" + tx.nearXidVersion() + + ", dhtTxId=" + tx.xidVersion() + + ", node=" + nearMapping.primary().id() + + ", err=" + e + ']'); + } + + fut.onResult(e); + } + } + }); + + if (sync) + hasMini = true; } } - return res; + return hasMini; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java index 2e19df2911509..d422fe7cb2b17 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java @@ -37,6 +37,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareResponse; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxCommitFuture; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedException; @@ -44,11 +45,9 @@ import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException; import org.apache.ignite.internal.util.tostring.GridToStringBuilder; import org.apache.ignite.internal.util.tostring.GridToStringExclude; -import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.transactions.TransactionConcurrency; import org.apache.ignite.transactions.TransactionIsolation; @@ -433,11 +432,9 @@ private void finishTx(boolean commit, @Nullable IgniteInternalFuture prepFut, Gr */ final IgniteInternalFuture finalPrepFut = prepFut; - lockFut.listen(new IgniteInClosure>() { - @Override public void apply(IgniteInternalFuture ignored) { - finishTx(false, finalPrepFut, fut); - } - }); + lockFut.listen(f -> cctx.tm().finisher().execute(this, () -> { + finishTx(false, finalPrepFut, fut); + })); return; } @@ -461,11 +458,48 @@ private void finishTx(boolean commit, @Nullable IgniteInternalFuture prepFut, Gr if (prepFut != null) prepFut.get(); // Check for errors. - boolean finished = localFinish(commit, false); + IgniteTxCommitFuture commitFuture = startLocalCommit(commit, false); + + if (!commitFuture.started()) { + commitFuture.get(); // Check for errors. - if (!finished) err = new IgniteCheckedException("Failed to finish transaction [commit=" + commit + ", tx=" + CU.txString(this) + ']'); + } + else { + if (commitFuture.async()) + commitFuture.listen(f -> cctx.tm().finisher().execute(this, () -> finishAsync(fut, primarySync, commit, commitFuture))); + else + finishAsync(fut, primarySync, commit, commitFuture); + + return; + } + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to finish transaction [commit=" + commit + ", tx=" + this + ']', e); + + err = e; + } + catch (Throwable t) { + fut.onDone(t); + + throw t; + } + + if (primarySync) + sendFinishReply(err); + + if (err != null) + fut.rollbackOnError(err); + else + fut.finish(commit); + } + + private void finishAsync(GridDhtTxFinishFuture fut, boolean primarySync, boolean commit, IgniteTxCommitFuture commitFuture) { + IgniteCheckedException err = null; + + try { + finishLocalCommit(commitFuture, commit, false); } catch (IgniteCheckedException e) { U.error(log, "Failed to finish transaction [commit=" + commit + ", tx=" + this + ']', e); @@ -500,21 +534,12 @@ public IgniteInternalFuture commitDhtLocalAsync() { GridDhtTxPrepareFuture prep = prepFut; - if (prep != null) { - if (prep.isDone()) - finishTx(true, prep, fut); - else { - prep.listen(new CI1>() { - @Override public void apply(IgniteInternalFuture f) { - finishTx(true, f, fut); - } - }); - } - } + if (prep != null) + prep.listen(f -> cctx.tm().finisher().execute(this, () -> finishTx(true, prep, fut))); else { assert optimistic(); - finishTx(true, null, fut); + cctx.tm().finisher().execute(this, () -> finishTx(true, null, fut)); } return fut; @@ -553,14 +578,10 @@ public IgniteInternalFuture rollbackDhtLocalAsync() { if (prepFut != null) { prepFut.complete(); - prepFut.listen(new CI1>() { - @Override public void apply(IgniteInternalFuture f) { - finishTx(false, f, fut); - } - }); + prepFut.listen(f -> cctx.tm().finisher().execute(this, () -> finishTx(false, prepFut, fut))); } else - finishTx(false, null, fut); + cctx.tm().finisher().execute(this, () -> finishTx(false, null, fut)); return fut; } @@ -572,7 +593,7 @@ public IgniteInternalFuture rollbackDhtLocalAsync() { /** {@inheritDoc} */ @SuppressWarnings({"CatchGenericClass", "ThrowableInstanceNeverThrown"}) - @Override public boolean localFinish(boolean commit, boolean clearThreadMap) throws IgniteCheckedException { + @Override public IgniteTxCommitFuture startLocalCommit(boolean commit, boolean clearThreadMap) throws IgniteCheckedException { assert nearFinFutId != null || isInvalidate() || !commit || isSystemInvalidate() || onePhaseCommit() || state() == PREPARED : "Invalid state [nearFinFutId=" + nearFinFutId + ", isInvalidate=" + isInvalidate() + ", commit=" + commit + @@ -580,7 +601,7 @@ public IgniteInternalFuture rollbackDhtLocalAsync() { assert nearMiniId != 0; - return super.localFinish(commit, clearThreadMap); + return super.startLocalCommit(commit, clearThreadMap); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java index 604fe0655801e..c5ed8849efc38 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java @@ -25,6 +25,7 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cluster.ClusterNode; @@ -39,6 +40,8 @@ import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxMapping; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareResponse; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxCommitEntriesFuture; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxCommitFuture; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; @@ -59,7 +62,6 @@ import org.apache.ignite.transactions.TransactionIsolation; import org.apache.ignite.transactions.TransactionState; import org.jetbrains.annotations.Nullable; -import java.util.concurrent.ConcurrentHashMap; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.NOOP; import static org.apache.ignite.transactions.TransactionState.COMMITTED; @@ -755,7 +757,7 @@ private IgniteInternalFuture obtainLockAsync( /** {@inheritDoc} */ @SuppressWarnings({"CatchGenericClass", "ThrowableInstanceNeverThrown"}) - @Override public boolean localFinish(boolean commit, boolean clearThreadMap) throws IgniteCheckedException { + @Override public IgniteTxCommitFuture startLocalCommit(boolean commit, boolean clearThreadMap) throws IgniteCheckedException { if (log.isDebugEnabled()) log.debug("Finishing dht local tx [tx=" + this + ", commit=" + commit + "]"); @@ -773,7 +775,7 @@ private IgniteInternalFuture obtainLockAsync( if (log.isDebugEnabled()) log.debug("Invalid transaction state for commit (another thread is committing): " + this); - return false; + return new IgniteTxCommitFuture(false); } } } @@ -782,19 +784,48 @@ private IgniteInternalFuture obtainLockAsync( if (log.isDebugEnabled()) log.debug("Invalid transaction state for rollback [state=" + state() + ", tx=" + this + ']'); - return false; + return new IgniteTxCommitFuture(false); } } + log.warning("Finish thread id = " + Thread.currentThread().getName()); + + if (!Thread.currentThread().getName().contains("dedicated")) { + throw new AssertionError("Commit requested not from dedicated stipe"); + } + + if (commit && !isRollbackOnly()) { + IgniteTxCommitEntriesFuture fut = startCommit(); + + IgniteTxCommitFuture commitFuture = new IgniteTxCommitFuture(fut, true); + + fut.listen(f -> commitFuture.onDone()); + + return commitFuture; + } + else + return new IgniteTxCommitFuture(userRollback(clearThreadMap), false); + } + + public void finishLocalCommit( + IgniteTxCommitFuture commitFuture, + boolean commit, + boolean clearThreadMap + ) throws IgniteCheckedException { + assert commitFuture.isDone(); + + if (!commitFuture.started()) + return; + IgniteCheckedException err = null; // Commit to DB first. This way if there is a failure, transaction // won't be committed. try { + commitFuture.get(); // Check for errors. + if (commit && !isRollbackOnly()) - userCommit(); - else - userRollback(clearThreadMap); + finishCommit(commitFuture.commitEntriesFuture()); } catch (IgniteCheckedException e) { err = e; @@ -834,8 +865,6 @@ private IgniteInternalFuture obtainLockAsync( } } } - - return true; } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java index ede8a4ec39484..ff8fb51246989 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java @@ -44,6 +44,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishResponse; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxCommitFuture; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException; @@ -56,7 +57,6 @@ import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.transactions.TransactionRollbackException; @@ -318,7 +318,14 @@ else if (err != null) err = new TransactionRollbackException("Failed to commit transaction.", err); try { - tx.localFinish(err == null, true); + IgniteTxCommitFuture commitFuture = tx.startLocalCommit(false, true); + + assert !commitFuture.async() : "Commit future should be synchronous in case of un-commit"; + + if (commitFuture.started()) + tx.finishLocalCommit(commitFuture, false, true); + else + commitFuture.get(); // Check for errors. } catch (IgniteCheckedException e) { if (err != null) @@ -408,6 +415,8 @@ public void finish(final boolean commit, final boolean clearThreadMap, final boo // finish request to it, so we can mark future as initialized. markInitialized(); + log.warning("One-phase commit -> " + tx); + return; } @@ -442,18 +451,16 @@ private void tryRollbackAsync(boolean onTimeout) { } } - curFut.listen(new IgniteInClosure>() { - @Override public void apply(IgniteInternalFuture fut) { - try { - fut.get(); + curFut.listen(f -> cctx.tm().finisher().execute(tx, () -> { + try { + curFut.get(); - tryRollbackAsync(onTimeout); - } - catch (IgniteCheckedException e) { - doFinish(false, false); - } + tryRollbackAsync(onTimeout); } - }); + catch (IgniteCheckedException e) { + doFinish(false, false); + } + })); } /** @@ -463,8 +470,33 @@ private void tryRollbackAsync(boolean onTimeout) { * @param clearThreadMap Clear thread map. */ private void doFinish(boolean commit, boolean clearThreadMap) { + IgniteTxCommitFuture commitFuture = tx.startLocalCommit(commit, clearThreadMap); + + log.warning("Local commit started " + cctx.tm().finisher().order(tx) + " " + commitFuture + " " + commitFuture.started() + " " + commitFuture.async() + " " + commitFuture.commitEntriesFuture()); + + if (commitFuture.async()) + commitFuture.listen(f -> cctx.tm().finisher().execute(tx, () -> doFinishAsync(commitFuture, commit, clearThreadMap))); + else + doFinishAsync(commitFuture, commit, clearThreadMap); + } + + /** + * Finishes a transaction. + * + * @param commit Commit. + * @param clearThreadMap Clear thread map. + */ + private void doFinishAsync(IgniteTxCommitFuture commitFuture, boolean commit, boolean clearThreadMap) { try { - if (tx.localFinish(commit, clearThreadMap) || (!commit && tx.state() == UNKNOWN)) { + if (commitFuture.started()) { + tx.finishLocalCommit(commitFuture, commit, clearThreadMap); + + log.warning("Local commit finished " + cctx.tm().finisher().order(tx) + " " + commitFuture + " " + commitFuture.started() + " " + commitFuture.async() + " " + commitFuture.commitEntriesFuture()); + } + else + commitFuture.get(); // Check for errors. + + if (commitFuture.started() || (!commit && tx.state() == UNKNOWN)) { if ((tx.onePhaseCommit() && needFinishOnePhase(commit)) || (!tx.onePhaseCommit() && mappings != null)) { if (mappings.single()) { GridDistributedTxMapping mapping = mappings.singleMapping(); @@ -494,6 +526,8 @@ private void doFinish(boolean commit, boolean clearThreadMap) { onDone(e); } finally { + cctx.tm().finisher().finishSend(tx); + if (commit && tx.onePhaseCommit() && !tx.writeMap().isEmpty()) // Readonly operations require no ack. @@ -636,34 +670,38 @@ private void checkBackup() { else { GridDhtTxFinishRequest finishReq = checkCommittedRequest(mini.futureId(), false); - try { - cctx.io().send(backup, finishReq, tx.ioPolicy()); + cctx.tm().finisher().send(tx, () -> { + try { + cctx.io().send(backup, finishReq, tx.ioPolicy()); - if (msgLog.isDebugEnabled()) { - msgLog.debug("Near finish fut, sent check committed request [" + - "txId=" + tx.nearXidVersion() + - ", node=" + backup.id() + ']'); + if (msgLog.isDebugEnabled()) { + msgLog.debug("Near finish fut, sent check committed request [" + + "txId=" + tx.nearXidVersion() + + ", node=" + backup.id() + ']'); + } } - } - catch (ClusterTopologyCheckedException ignored) { - mini.onNodeLeft(backupId, false); - } - catch (IgniteCheckedException e) { - if (msgLog.isDebugEnabled()) { - msgLog.debug("Near finish fut, failed to send check committed request [" + - "txId=" + tx.nearXidVersion() + - ", node=" + backup.id() + - ", err=" + e + ']'); + catch (ClusterTopologyCheckedException ignored) { + mini.onNodeLeft(backupId, false); } + catch (IgniteCheckedException e) { + if (msgLog.isDebugEnabled()) { + msgLog.debug("Near finish fut, failed to send check committed request [" + + "txId=" + tx.nearXidVersion() + + ", node=" + backup.id() + + ", err=" + e + ']'); + } - mini.onDone(e); - } + mini.onDone(e); + } + }); } } } else readyNearMappingFromBackup(mapping); } + + cctx.tm().finisher().finishSend(tx); } /** @@ -752,10 +790,7 @@ private void finish(int miniId, GridDistributedTxMapping m, boolean commit, bool assert !m.empty() : m + " " + tx.state(); - CacheWriteSynchronizationMode syncMode = tx.syncMode(); - - if (m.explicitLock()) - syncMode = FULL_SYNC; + final CacheWriteSynchronizationMode syncMode = m.explicitLock() ? FULL_SYNC : tx.syncMode(); GridNearTxFinishRequest req = new GridNearTxFinishRequest( futId, @@ -798,39 +833,44 @@ private void finish(int miniId, GridDistributedTxMapping m, boolean commit, bool if (tx.pessimistic() && !useCompletedVer) cctx.tm().beforeFinishRemote(n.id(), tx.threadId()); - try { - cctx.io().send(n, req, tx.ioPolicy()); + cctx.tm().finisher().send(tx, () -> { + try { + cctx.io().send(n, req, tx.ioPolicy()); - if (msgLog.isDebugEnabled()) { - msgLog.debug("Near finish fut, sent request [" + - "txId=" + tx.nearXidVersion() + - ", node=" + n.id() + ']'); - } + if (msgLog.isDebugEnabled()) { + msgLog.debug("Near finish fut, sent request [" + + "txId=" + tx.nearXidVersion() + + ", node=" + n.id() + ']'); + } - boolean wait = syncMode != FULL_ASYNC; + boolean wait = syncMode != FULL_ASYNC; - // If we don't wait for result, then mark future as done. - if (!wait) - fut.onDone(); - } - catch (ClusterTopologyCheckedException ignored) { - // Remove previous mapping. - mappings.remove(m.primary().id()); + // If we don't wait for result, then mark future as done. + if (!wait) + fut.onDone(); + } + catch (ClusterTopologyCheckedException ignored) { + // Remove previous mapping. + mappings.remove(m.primary().id()); - fut.onNodeLeft(n.id(), false); - } - catch (IgniteCheckedException e) { - if (msgLog.isDebugEnabled()) { - msgLog.debug("Near finish fut, failed to send request [" + - "txId=" + tx.nearXidVersion() + - ", node=" + n.id() + - ", err=" + e + ']'); + fut.onNodeLeft(n.id(), false); } + catch (IgniteCheckedException e) { + if (msgLog.isDebugEnabled()) { + msgLog.debug("Near finish fut, failed to send request [" + + "txId=" + tx.nearXidVersion() + + ", node=" + n.id() + + ", err=" + e + ']'); + } - // Fail the whole thing. - fut.onDone(e); - } + // Fail the whole thing. + fut.onDone(e); + } + }); } + + if (miniId == mappings.mappings().size()) + cctx.tm().finisher().finishSend(tx); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java index 65e1ea414661e..2eedb6db6c09c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java @@ -47,10 +47,10 @@ import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate; -import org.apache.ignite.internal.processors.cache.GridCacheVersionedFuture; import org.apache.ignite.internal.processors.cache.GridCacheOperation; import org.apache.ignite.internal.processors.cache.GridCacheReturn; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.GridCacheVersionedFuture; import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedCacheEntry; @@ -62,6 +62,8 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtDetachedCacheEntry; import org.apache.ignite.internal.processors.cache.dr.GridCacheDrInfo; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxCommitEntriesFuture; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxCommitFuture; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; import org.apache.ignite.internal.processors.cache.transactions.TransactionProxy; @@ -116,7 +118,6 @@ import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK; import static org.apache.ignite.transactions.TransactionState.ROLLING_BACK; import static org.apache.ignite.transactions.TransactionState.SUSPENDED; -import static org.apache.ignite.transactions.TransactionState.UNKNOWN; /** * Replicated user transaction. @@ -3147,86 +3148,57 @@ private void readyNearLock(IgniteTxEntry txEntry, /** {@inheritDoc} */ @SuppressWarnings({"CatchGenericClass", "ThrowableInstanceNeverThrown"}) - @Override public boolean localFinish(boolean commit, boolean clearThreadMap) throws IgniteCheckedException { + @Override public IgniteTxCommitFuture startLocalCommit(boolean commit, boolean clearThreadMap) { if (log.isDebugEnabled()) log.debug("Finishing near local tx [tx=" + this + ", commit=" + commit + "]"); - if (commit) { - if (!state(COMMITTING)) { - TransactionState state = state(); + try { + if (commit) { + if (!state(COMMITTING)) { + TransactionState state = state(); - if (state != COMMITTING && state != COMMITTED) - throw isRollbackOnly() ? timedOut() ? timeoutException() : rollbackException() : - new IgniteCheckedException("Invalid transaction state for commit [state=" + state() + - ", tx=" + this + ']'); - else { - if (log.isDebugEnabled()) - log.debug("Invalid transaction state for commit (another thread is committing): " + this); + if (state != COMMITTING && state != COMMITTED) + throw isRollbackOnly() ? timedOut() ? timeoutException() : rollbackException() : + new IgniteCheckedException("Invalid transaction state for commit [state=" + state() + + ", tx=" + this + ']'); + else { + if (log.isDebugEnabled()) + log.debug("Invalid transaction state for commit (another thread is committing): " + this); - return false; + return new IgniteTxCommitFuture(false); + } } } - } - else { - if (!state(ROLLING_BACK)) { - if (log.isDebugEnabled()) - log.debug("Invalid transaction state for rollback [state=" + state() + ", tx=" + this + ']'); + else { + if (!state(ROLLING_BACK)) { + if (log.isDebugEnabled()) + log.debug("Invalid transaction state for rollback [state=" + state() + ", tx=" + this + ']'); - return false; + return new IgniteTxCommitFuture(false); + } } } - - IgniteCheckedException err = null; - - // Commit to DB first. This way if there is a failure, transaction - // won't be committed. - try { - if (commit && !isRollbackOnly()) - userCommit(); - else - userRollback(clearThreadMap); + catch (Throwable t) { + return new IgniteTxCommitFuture(t); } - catch (IgniteCheckedException e) { - err = e; - - commit = false; - - // If heuristic error. - if (!isRollbackOnly()) { - invalidate = true; - systemInvalidate(true); + log.warning("Finish thread id = " + Thread.currentThread().getName()); - U.warn(log, "Set transaction invalidation flag to true due to error [tx=" + this + ", err=" + err + ']'); - } + if (!Thread.currentThread().getName().contains("dedicated")) { + throw new AssertionError("Commit requested not from dedicated stipe"); } - if (err != null) { - state(UNKNOWN); + if (commit && !isRollbackOnly()) { + IgniteTxCommitEntriesFuture fut = startCommit(); - throw err; - } - else { - // Committed state will be set in finish future onDone callback. - if (commit) { - if (!onePhaseCommit()) { - if (!state(COMMITTED)) { - state(UNKNOWN); + IgniteTxCommitFuture commitFuture = new IgniteTxCommitFuture(fut, true); - throw new IgniteCheckedException("Invalid transaction state for commit: " + this); - } - } - } - else { - if (!state(ROLLED_BACK)) { - state(UNKNOWN); + fut.listen(f -> commitFuture.onDone()); - throw new IgniteCheckedException("Invalid transaction state for rollback: " + this); - } - } + return commitFuture; } - - return true; + else + return new IgniteTxCommitFuture(userRollback(clearThreadMap), false); } /** @@ -3332,29 +3304,27 @@ public IgniteInternalFuture commitNearTxLocalAsync() { final IgniteInternalFuture prepareFut = prepareNearTxLocal(); - prepareFut.listen(new CI1>() { - @Override public void apply(IgniteInternalFuture f) { - try { - // Make sure that here are no exceptions. - prepareFut.get(); + prepareFut.listen(f -> cctx.tm().finisher().execute(this, () -> { + try { + // Make sure that here are no exceptions. + prepareFut.get(); - fut0.finish(true, true, false); - } - catch (Error | RuntimeException e) { - COMMIT_ERR_UPD.compareAndSet(GridNearTxLocal.this, null, e); + fut0.finish(true, true, false); + } + catch (Error | RuntimeException e) { + COMMIT_ERR_UPD.compareAndSet(GridNearTxLocal.this, null, e); - fut0.finish(false, true, false); + fut0.finish(false, true, false); - throw e; - } - catch (IgniteCheckedException e) { - COMMIT_ERR_UPD.compareAndSet(GridNearTxLocal.this, null, e); + throw e; + } + catch (IgniteCheckedException e) { + COMMIT_ERR_UPD.compareAndSet(GridNearTxLocal.this, null, e); - if (!(e instanceof NodeStoppingException)) - fut0.finish(false, true, true); - } + if (!(e instanceof NodeStoppingException)) + fut0.finish(false, true, true); } - }); + })); return fut0; } @@ -3424,35 +3394,19 @@ public IgniteInternalFuture rollbackNearTxLocalAsync(final boo cctx.mvcc().addFuture(fut0, fut0.futureId()); - if (prepFut == null || prepFut.isDone()) { + final IgniteInternalFuture prepareFuture = prepFut == null ? new GridFinishedFuture<>() : prepFut; + + prepareFuture.listen(f -> cctx.tm().finisher().execute(this, () -> { try { // Check for errors in prepare future. - if (prepFut != null) - prepFut.get(); - } - catch (IgniteCheckedException e) { + prepareFuture.get(); + } catch (IgniteCheckedException e) { if (log.isDebugEnabled()) log.debug("Got optimistic tx failure [tx=" + this + ", err=" + e + ']'); } fut0.finish(false, clearThreadMap, onTimeout); - } - else { - prepFut.listen(new CI1>() { - @Override public void apply(IgniteInternalFuture f) { - try { - // Check for errors in prepare future. - f.get(); - } - catch (IgniteCheckedException e) { - if (log.isDebugEnabled()) - log.debug("Got optimistic tx failure [tx=" + this + ", err=" + e + ']'); - } - - fut0.finish(false, clearThreadMap, onTimeout); - } - }); - } + })); return fut0; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxCommitEntriesFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxCommitEntriesFuture.java new file mode 100644 index 0000000000000..bd868875ee846 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxCommitEntriesFuture.java @@ -0,0 +1,25 @@ +package org.apache.ignite.internal.processors.cache.transactions; + +import org.apache.ignite.internal.processors.cache.GridCacheUpdateTxResult; +import org.apache.ignite.internal.util.future.GridCompoundFuture; + +public class IgniteTxCommitEntriesFuture extends GridCompoundFuture { + + public static final IgniteTxCommitEntriesFuture FINISHED = finished(); + + public static IgniteTxCommitEntriesFuture finished() { + IgniteTxCommitEntriesFuture fut = new IgniteTxCommitEntriesFuture(); + + fut.onDone(); + + return fut; + } + + public static IgniteTxCommitEntriesFuture finishedWithError(Throwable t) { + IgniteTxCommitEntriesFuture fut = new IgniteTxCommitEntriesFuture(); + + fut.onDone(t); + + return fut; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxCommitFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxCommitFuture.java new file mode 100644 index 0000000000000..6115af00662fa --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxCommitFuture.java @@ -0,0 +1,44 @@ +package org.apache.ignite.internal.processors.cache.transactions; + +import org.apache.ignite.internal.util.future.GridFutureAdapter; + +public class IgniteTxCommitFuture extends GridFutureAdapter { + private IgniteTxCommitEntriesFuture commitEntriesFuture; + + private boolean started; + + private boolean async; + + public IgniteTxCommitFuture(IgniteTxCommitEntriesFuture commitEntriesFuture, boolean async) { + this.commitEntriesFuture = commitEntriesFuture; + this.async = async; + this.started = true; + + if (!async) + onDone(); + } + + public IgniteTxCommitFuture(Throwable err) { + onDone(err); + + started = false; + } + + public IgniteTxCommitFuture(boolean started) { + onDone(); + + this.started = started; + } + + public IgniteTxCommitEntriesFuture commitEntriesFuture() { + return commitEntriesFuture; + } + + public boolean async() { + return async; + } + + public boolean started() { + return started; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java new file mode 100644 index 0000000000000..2b2fad927a341 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java @@ -0,0 +1,116 @@ +package org.apache.ignite.internal.processors.cache.transactions; + +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.jsr166.ConcurrentLinkedHashMap; + +/** + * + */ +public class IgniteTxFinisher { + private static final int DEDICATED_WORKER_IDX = 0; + + private final IgniteLogger log; + + private final GridCacheSharedContext cctx; + + private final ConcurrentLinkedHashMap txOrdering = new ConcurrentLinkedHashMap(); + + private final ConcurrentLinkedHashMap txs = new ConcurrentLinkedHashMap<>(); + + private final Map> delayedSendings = new LinkedHashMap<>(); + + private long ordered; + + private long send; + + public IgniteTxFinisher(GridCacheSharedContext cctx) { + this.cctx = cctx; + this.log = cctx.logger(getClass()); + } + + public void execute(IgniteTxAdapter tx, Runnable transactionOp) { + cctx.kernalContext().getStripedExecutorService().executeDedicated(DEDICATED_WORKER_IDX, () -> { + GridCacheVersion txId = tx.xidVersion(); + + if (txId != null) { + boolean isNew = !txOrdering.containsKey(txId); + + if (!txOrdering.containsKey(txId)) { + txOrdering.put(txId, ordered); + + txs.put(ordered, tx); + + ordered++; + } + + //if (isNew) + //log.warning("New tx = " + (ordered - 1) + " " + tx); + } + + transactionOp.run(); + }); + } + + public void send(IgniteTxAdapter tx, Runnable transactionSendOp) { + GridCacheVersion txId = tx.xidVersion(); + + long order = txOrdering.get(txId); + + //log.warning("Try send " + order + " " + tx + ". Already sent: " + send); + + // Allowed to immediately send tx. + if (send >= order) { + transactionSendOp.run(); + + //log.warning("Send directly -> " + order + " " + tx); + } + else { + delayedSendings.computeIfAbsent(order, o -> new ArrayList<>()).add(transactionSendOp); + + //log.warning("Send delayed -> " + order + " " + tx); + } + } + + public void finishSend(IgniteTxAdapter tx) { + GridCacheVersion txId = tx.xidVersion(); + + long order = txOrdering.get(txId); + + if (order >= send) + if (!delayedSendings.containsKey(order)) + delayedSendings.put(order, new ArrayList<>()); + + //log.warning("Finish send: " + order + " " + tx + " Already sent: " + send); + + if (order == send) { + delayedSendings.remove(send); + txs.remove(send); + + send++; + + //log.warning("Send incremented: " + send); + + while (delayedSendings.containsKey(send)) { + log.warning("Send delayed 2 - > " + send); + + List delayed = delayedSendings.remove(send); + txs.remove(send); + + for (Runnable sendClj : delayed) + sendClj.run(); + + send++; + } + } + } + + public long order(IgniteTxAdapter tx) { + return txOrdering.get(tx.xidVersion()); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java index bf2c729720bfb..eb97efd03e386 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java @@ -188,7 +188,9 @@ public IgniteTxHandler(GridCacheSharedContext ctx) { ctx.io().addCacheHandler(0, GridNearTxFinishRequest.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { - processNearTxFinishRequest(nodeId, (GridNearTxFinishRequest)msg); + GridNearTxFinishRequest finishReqMsg = (GridNearTxFinishRequest) msg; + + processNearTxFinishRequest(nodeId, finishReqMsg); } }); @@ -212,7 +214,9 @@ public IgniteTxHandler(GridCacheSharedContext ctx) { ctx.io().addCacheHandler(0, GridDhtTxFinishRequest.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { - processDhtTxFinishRequest(nodeId, (GridDhtTxFinishRequest)msg); + GridDhtTxFinishRequest finishReqMsg = (GridDhtTxFinishRequest) msg; + + processDhtTxFinishRequest(nodeId, finishReqMsg); } }); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java index 9c381232ff350..4f1f17c0e0fb5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java @@ -18,7 +18,6 @@ package org.apache.ignite.internal.processors.cache.transactions; import java.io.Externalizable; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -50,7 +49,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; import org.apache.ignite.internal.processors.cache.GridCacheFilterFailedException; -import org.apache.ignite.internal.processors.cache.GridCacheIndexUpdateException; import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate; import org.apache.ignite.internal.processors.cache.GridCacheOperation; import org.apache.ignite.internal.processors.cache.GridCacheReturn; @@ -478,44 +476,43 @@ protected GridCacheEntryEx entryEx(GridCacheContext cacheCtx, IgniteTxKey key, A /** {@inheritDoc} */ @SuppressWarnings({"CatchGenericClass"}) - @Override public void userCommit() throws IgniteCheckedException { + @Override public IgniteTxCommitEntriesFuture startCommit() { TransactionState state = state(); - if (state != COMMITTING) { - if (remainingTime() == -1) - throw new IgniteTxTimeoutCheckedException("Transaction timed out: " + this); + try { + if (state != COMMITTING) { + if (remainingTime() == -1) + throw new IgniteTxTimeoutCheckedException("Transaction timed out: " + this); - setRollbackOnly(); + setRollbackOnly(); - throw new IgniteCheckedException("Invalid transaction state for commit [state=" + state + ", tx=" + this + ']'); - } + throw new IgniteCheckedException("Invalid transaction state for commit [state=" + state + ", tx=" + this + ']'); + } - checkValid(); + checkValid(); - Collection commitEntries = (near() || cctx.snapshot().needTxReadLogging()) ? allEntries() : writeEntries(); + Collection commitEntries = commitEntries(); - boolean empty = F.isEmpty(commitEntries); + boolean empty = F.isEmpty(commitEntries); - // Register this transaction as completed prior to write-phase to - // ensure proper lock ordering for removed entries. - // We add colocated transaction to committed set even if it is empty to correctly order - // locks on backup nodes. - if (!empty || colocated()) - cctx.tm().addCommittedTx(this); + // Register this transaction as completed prior to write-phase to + // ensure proper lock ordering for removed entries. + // We add colocated transaction to committed set even if it is empty to correctly order + // locks on backup nodes. + if (!empty || colocated()) + cctx.tm().addCommittedTx(this); - if (!empty) { - batchStoreCommit(writeEntries()); + if (empty) + return IgniteTxCommitEntriesFuture.FINISHED; - WALPointer ptr = null; + batchStoreCommit(writeEntries()); - cctx.database().checkpointReadLock(); + AffinityTopologyVersion topVer = topologyVersion(); try { cctx.tm().txContext(this); - AffinityTopologyVersion topVer = topologyVersion(); - - GridCompoundFuture entriesCommitFuture = new GridCompoundFuture<>(); + IgniteTxCommitEntriesFuture commitEntriesFuture = new IgniteTxCommitEntriesFuture(); /* * Commit to cache. Note that for 'near' transaction we loop through all the entries. @@ -535,7 +532,7 @@ protected GridCacheEntryEx entryEx(GridCacheContext cacheCtx, IgniteTxKey key, A // Must try to evict near entries before committing from // transaction manager to make sure locks are held. if (evictNearEntry(txEntry, false)) { - entriesCommitFuture.add(new GridFinishedFuture<>()); + commitEntriesFuture.add(new GridFinishedFuture<>()); continue; } @@ -543,13 +540,13 @@ protected GridCacheEntryEx entryEx(GridCacheContext cacheCtx, IgniteTxKey key, A if (cacheCtx.isNear() && cacheCtx.dr().receiveEnabled()) { cached.markObsolete(xidVer); - entriesCommitFuture.add(new GridFinishedFuture<>()); + commitEntriesFuture.add(new GridFinishedFuture<>()); continue; } if (cached.detached()) { - entriesCommitFuture.add(new GridFinishedFuture<>()); + commitEntriesFuture.add(new GridFinishedFuture<>()); continue; } @@ -863,95 +860,117 @@ assert ownsLock(txEntry.cached()): assert operationFuture != null : "Entry operation must be submitted"; - entriesCommitFuture.add(operationFuture); + commitEntriesFuture.add(operationFuture); } catch (Throwable ex) { - entriesCommitFuture.onDone(ex); + commitEntriesFuture.onDone(ex); break; } } - entriesCommitFuture.markInitialized(); + commitEntriesFuture.markInitialized(); - try { - entriesCommitFuture.get(); + return commitEntriesFuture; + } + finally { + cctx.tm().resetContext(); + } + } + catch (Throwable t) { + return IgniteTxCommitEntriesFuture.finishedWithError(t); + } + } + + private Collection commitEntries() { + return (near() || cctx.snapshot().needTxReadLogging()) ? allEntries() : writeEntries(); + } + + public void finishCommit(IgniteTxCommitEntriesFuture commitEntriesFuture) throws IgniteCheckedException { + assert commitEntriesFuture.isDone() : "Entries commit future must be done before finish commit: " + commitEntriesFuture; - List> futures = entriesCommitFuture.futures(); + WALPointer latestPtr = null; - assert commitEntries.size() == futures.size(); + cctx.tm().txContext(this); - int txEntryIdx = 0; + try { + try { + Collection commitEntries = commitEntries(); - for (IgniteTxEntry txEntry : commitEntries) { - GridCacheUpdateTxResult txResult = futures.get(txEntryIdx).get(); + commitEntriesFuture.get(); - if (txResult != null && txResult.success()) { - if (txResult.updatePartitionCounter() != -1) - txEntry.updateCounter(txResult.updatePartitionCounter()); + List> futures = commitEntriesFuture.futures(); - WALPointer loggedPtr = txResult.loggedPointer(); + assert commitEntries.size() == futures.size(); - // Find latest logged WAL pointer. - if (loggedPtr != null) - if (ptr == null || loggedPtr.compareTo(ptr) > 0) - ptr = loggedPtr; - } + int txEntryIdx = 0; - txEntryIdx++; - } - } - catch (Throwable ex) { - // We are about to initiate transaction rollback when tx has started to committing. - // Need to remove version from committed list. - cctx.tm().removeCommittedTx(this); + for (IgniteTxEntry txEntry : commitEntries) { + GridCacheUpdateTxResult txResult = futures.get(txEntryIdx).get(); - boolean hasInvalidEnvironmentIssue = X.hasCause(ex, InvalidEnvironmentException.class); + if (txResult != null && txResult.success()) { + if (txResult.updatePartitionCounter() != -1) + txEntry.updateCounter(txResult.updatePartitionCounter()); - IgniteCheckedException err = new IgniteTxHeuristicCheckedException("Failed to locally write to cache " + - "(all transaction entries will be invalidated, however there was a window when " + - "entries for this transaction were visible to others): " + this, ex); + WALPointer loggedPtr = txResult.loggedPointer(); - if (hasInvalidEnvironmentIssue) { - U.warn(log, "Failed to commit transaction, node is stopping " + - "[tx=" + this + ", err=" + ex + ']'); + // Find latest logged WAL pointer. + if (loggedPtr != null) + if (latestPtr == null || loggedPtr.compareTo(latestPtr) > 0) + latestPtr = loggedPtr; } - else - U.error(log, "Heuristic transaction failure.", err); - COMMIT_ERR_UPD.compareAndSet(this, null, err); + txEntryIdx++; + } + } + catch (Throwable ex) { + // We are about to initiate transaction rollback when tx has started to committing. + // Need to remove version from committed list. + cctx.tm().removeCommittedTx(this); - state(UNKNOWN); + boolean hasInvalidEnvironmentIssue = X.hasCause(ex, InvalidEnvironmentException.class); - try { - // Courtesy to minimize damage. - uncommit(hasInvalidEnvironmentIssue); - } - catch (Throwable ex1) { - U.error(log, "Failed to uncommit transaction: " + this, ex1); + IgniteCheckedException err = new IgniteTxHeuristicCheckedException("Failed to locally write to cache " + + "(all transaction entries will be invalidated, however there was a window when " + + "entries for this transaction were visible to others): " + this, ex); - if (ex1 instanceof Error) - throw ex1; - } + if (hasInvalidEnvironmentIssue) { + U.warn(log, "Failed to commit transaction, node is stopping " + + "[tx=" + this + ", err=" + ex + ']'); + } + else + U.error(log, "Heuristic transaction failure.", err); + + COMMIT_ERR_UPD.compareAndSet(this, null, err); + + state(UNKNOWN); - if (ex instanceof Error) - throw ex; + try { + // Courtesy to minimize damage. + uncommit(hasInvalidEnvironmentIssue); + } + catch (Throwable ex1) { + U.error(log, "Failed to uncommit transaction: " + this, ex1); - throw err; + if (ex1 instanceof Error) + throw ex1; } - if (ptr != null && !cctx.tm().logTxRecords()) - cctx.wal().flush(ptr, false); - } - catch (StorageException e) { - throw new IgniteCheckedException("Failed to log transaction record " + - "(transaction will be rolled back): " + this, e); - } - finally { - cctx.database().checkpointReadUnlock(); + if (ex instanceof Error) + throw ex; - cctx.tm().resetContext(); + throw err; } + + if (latestPtr != null && !cctx.tm().logTxRecords()) + cctx.wal().flush(latestPtr, false); + } + catch (StorageException e) { + throw new IgniteCheckedException("Failed to log transaction record " + + "(transaction will be rolled back): " + this, e); + } + finally { + cctx.tm().resetContext(); } // Do not unlock transaction entries if one-phase commit. @@ -1067,39 +1086,46 @@ public Collection rolledbackVersions() { } /** {@inheritDoc} */ - @Override public void userRollback(boolean clearThreadMap) throws IgniteCheckedException { + @Override public IgniteTxCommitEntriesFuture userRollback(boolean clearThreadMap) { TransactionState state = state(); - if (state != ROLLING_BACK && state != ROLLED_BACK) { - setRollbackOnly(); + try { + if (state != ROLLING_BACK && state != ROLLED_BACK) { + setRollbackOnly(); - throw new IgniteCheckedException("Invalid transaction state for rollback [state=" + state + - ", tx=" + this + ']'); - } + throw new IgniteCheckedException("Invalid transaction state for rollback [state=" + state + + ", tx=" + this + ']'); + } - if (near()) { - // Must evict near entries before rolling back from - // transaction manager, so they will be removed from cache. - for (IgniteTxEntry e : allEntries()) - evictNearEntry(e, false); - } + if (near()) { + // Must evict near entries before rolling back from + // transaction manager, so they will be removed from cache. + for (IgniteTxEntry e : allEntries()) + evictNearEntry(e, false); + } - if (DONE_FLAG_UPD.compareAndSet(this, 0, 1)) { - cctx.tm().rollbackTx(this, clearThreadMap, false); + if (DONE_FLAG_UPD.compareAndSet(this, 0, 1)) { + cctx.tm().rollbackTx(this, clearThreadMap, false); - if (!internal()) { - Collection stores = txState.stores(cctx); + if (!internal()) { + Collection stores = txState.stores(cctx); - if (stores != null && !stores.isEmpty()) { - assert isWriteToStoreFromDhtValid(stores) : - "isWriteToStoreFromDht can't be different within one transaction"; + if (stores != null && !stores.isEmpty()) { + assert isWriteToStoreFromDhtValid(stores) : + "isWriteToStoreFromDht can't be different within one transaction"; - boolean isWriteToStoreFromDht = F.first(stores).isWriteToStoreFromDht(); + boolean isWriteToStoreFromDht = F.first(stores).isWriteToStoreFromDht(); - if (!stores.isEmpty() && (near() || isWriteToStoreFromDht)) - sessionEnd(stores, false); + if (!stores.isEmpty() && (near() || isWriteToStoreFromDht)) + sessionEnd(stores, false); + } } } + + return IgniteTxCommitEntriesFuture.FINISHED; + } + catch (Throwable t) { + return IgniteTxCommitEntriesFuture.finishedWithError(t); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java index b61b1a9a629b5..cf1f5b2712b9c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java @@ -39,13 +39,13 @@ public interface IgniteTxLocalEx extends IgniteInternalTx { /** * @throws IgniteCheckedException If commit failed. */ - public void userCommit() throws IgniteCheckedException; + public IgniteTxCommitEntriesFuture startCommit() throws IgniteCheckedException; /** * @param clearThreadMap If {@code true} removes {@link GridNearTxLocal} from thread map. * @throws IgniteCheckedException If rollback failed. */ - public void userRollback(boolean clearThreadMap) throws IgniteCheckedException; + public IgniteTxCommitEntriesFuture userRollback(boolean clearThreadMap) throws IgniteCheckedException; /** * Finishes transaction (either commit or rollback). @@ -55,5 +55,5 @@ public interface IgniteTxLocalEx extends IgniteInternalTx { * @return {@code True} if state has been changed. * @throws IgniteCheckedException If finish failed. */ - public boolean localFinish(boolean commit, boolean clearThreadMap) throws IgniteCheckedException; + public IgniteTxCommitFuture startLocalCommit(boolean commit, boolean clearThreadMap) throws IgniteCheckedException; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java index ffaaf439c8f42..030fbb7c63c58 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java @@ -209,6 +209,12 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter { /** Flag indicates that {@link TxRecord} records will be logged to WAL. */ private boolean logTxRecords; + private IgniteTxFinisher finisher; + + public IgniteTxFinisher finisher() { + return finisher; + } + /** {@inheritDoc} */ @Override protected void onKernalStop0(boolean cancel) { cctx.gridIO().removeMessageListener(TOPIC_TX); @@ -220,6 +226,8 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter { txHnd = new IgniteTxHandler(cctx); + finisher = new IgniteTxFinisher(cctx); + deferredAckMsgSnd = new GridDeferredAckMessageSender(cctx.time(), cctx.kernalContext().closure()) { @Override public int getTimeout() { return DEFERRED_ONE_PHASE_COMMIT_ACK_REQUEST_TIMEOUT; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java b/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java index 904b8d1a5f48b..4d41040e7bec3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java @@ -61,6 +61,9 @@ public class StripedExecutor implements ExecutorService { /** */ private final IgniteLogger log; + /** */ + private final int stripesCnt; + /** * @param cnt Count. * @param igniteInstanceName Node name. @@ -75,9 +78,10 @@ public StripedExecutor( String poolName, final IgniteLogger log, IgniteInClosure errHnd, - GridWorkerListener gridWorkerLsnr + GridWorkerListener gridWorkerLsnr, + int dedicated ) { - this(cnt, igniteInstanceName, poolName, log, errHnd, false, gridWorkerLsnr); + this(cnt, igniteInstanceName, poolName, log, errHnd, false, gridWorkerLsnr, dedicated); } /** @@ -96,15 +100,18 @@ public StripedExecutor( final IgniteLogger log, IgniteInClosure errHnd, boolean stealTasks, - GridWorkerListener gridWorkerLsnr + GridWorkerListener gridWorkerLsnr, + int dedicated ) { A.ensure(cnt > 0, "cnt > 0"); + stripesCnt = cnt; + boolean success = false; - stripes = new Stripe[cnt]; + stripes = new Stripe[cnt + dedicated]; - completedCntrs = new long[cnt]; + completedCntrs = new long[cnt + dedicated]; Arrays.fill(completedCntrs, -1); @@ -117,7 +124,10 @@ public StripedExecutor( : new StripeConcurrentQueue(igniteInstanceName, poolName, i, log, errHnd, gridWorkerLsnr); } - for (int i = 0; i < cnt; i++) + for (int i = cnt; i < cnt + dedicated; i++) + stripes[i] = new StripeConcurrentQueue(igniteInstanceName, poolName + "-dedicated", i, log, errHnd, gridWorkerLsnr); + + for (int i = 0; i < cnt + dedicated; i++) stripes[i].start(); success = true; @@ -181,7 +191,7 @@ public void checkStarvation() { * @return Stripes count. */ public int stripes() { - return stripes.length; + return stripesCnt; } /** @@ -196,10 +206,14 @@ public void execute(int idx, Runnable cmd) { else { assert idx >= 0 : idx; - stripes[idx % stripes.length].execute(cmd); + stripes[idx % stripesCnt].execute(cmd); } } + public void executeDedicated(int dedicatedIdx, Runnable cmd) { + stripes[stripesCnt + dedicatedIdx].execute(cmd); + } + /** {@inheritDoc} */ @Override public void shutdown() { signalStop(); @@ -207,7 +221,7 @@ public void execute(int idx, Runnable cmd) { /** {@inheritDoc} */ @Override public void execute(@NotNull Runnable cmd) { - stripes[ThreadLocalRandom.current().nextInt(stripes.length)].execute(cmd); + stripes[ThreadLocalRandom.current().nextInt(stripesCnt)].execute(cmd); } /** @@ -304,7 +318,7 @@ public long completedTasks() { * @return Completed tasks per stripe count. */ public long[] stripesCompletedTasks() { - long[] res = new long[stripes()]; + long[] res = new long[stripesCnt]; for (int i = 0; i < res.length; i++) res[i] = stripes[i].completedCnt; @@ -316,7 +330,7 @@ public long[] stripesCompletedTasks() { * @return Number of active tasks per stripe. */ public boolean[] stripesActiveStatuses() { - boolean[] res = new boolean[stripes()]; + boolean[] res = new boolean[stripesCnt]; for (int i = 0; i < res.length; i++) res[i] = stripes[i].active; @@ -342,7 +356,7 @@ public int activeStripesCount() { * @return Size of queue per stripe. */ public int[] stripesQueueSizes() { - int[] res = new int[stripes()]; + int[] res = new int[stripesCnt]; for (int i = 0; i < res.length; i++) res[i] = stripes[i].queueSize(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/StripedExecutorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/StripedExecutorTest.java index 9a4bf0619c4df..86c70ae36592d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/util/StripedExecutorTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/util/StripedExecutorTest.java @@ -33,7 +33,7 @@ public class StripedExecutorTest extends GridCommonAbstractTest { stripedExecSvc = new StripedExecutor(3, "foo name", "pool name", new JavaLogger(), new IgniteInClosure() { @Override public void apply(Throwable throwable) {} - }, null); + }, null, 0); } /** {@inheritDoc} */ From e6329578ca04bdf456ad06b42ee9abaee248dc8c Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Tue, 21 Aug 2018 17:39:09 +0300 Subject: [PATCH 05/18] IGNITE-9270 WIP --- .../near/GridNearTxFinishFuture.java | 65 +++++++++---------- .../cache/transactions/IgniteTxFinisher.java | 8 ++- 2 files changed, 37 insertions(+), 36 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java index ff8fb51246989..24cb2565b6d0a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java @@ -237,37 +237,39 @@ public void onResult(UUID nodeId, GridNearTxFinishResponse res) { */ public void onResult(UUID nodeId, GridDhtTxFinishResponse res) { if (!isDone()) { - boolean found = false; + cctx.tm().finisher().execute(tx, () -> { + boolean found = false; - for (IgniteInternalFuture fut : futures()) { - if (fut.getClass() == CheckBackupMiniFuture.class) { - CheckBackupMiniFuture f = (CheckBackupMiniFuture)fut; + for (IgniteInternalFuture fut : futures()) { + if (fut.getClass() == CheckBackupMiniFuture.class) { + CheckBackupMiniFuture f = (CheckBackupMiniFuture)fut; - if (f.futureId() == res.miniId()) { - found = true; + if (f.futureId() == res.miniId()) { + found = true; - assert f.node().id().equals(nodeId); + assert f.node().id().equals(nodeId); - if (res.returnValue() != null) - tx.implicitSingleResult(res.returnValue()); + if (res.returnValue() != null) + tx.implicitSingleResult(res.returnValue()); - f.onDhtFinishResponse(res); + f.onDhtFinishResponse(res); + } } - } - else if (fut.getClass() == CheckRemoteTxMiniFuture.class) { - CheckRemoteTxMiniFuture f = (CheckRemoteTxMiniFuture)fut; + else if (fut.getClass() == CheckRemoteTxMiniFuture.class) { + CheckRemoteTxMiniFuture f = (CheckRemoteTxMiniFuture)fut; - if (f.futureId() == res.miniId()) - f.onDhtFinishResponse(nodeId, false); + if (f.futureId() == res.miniId()) + f.onDhtFinishResponse(nodeId, false); + } } - } - if (!found && msgLog.isDebugEnabled()) { - msgLog.debug("Near finish fut, failed to find mini future [txId=" + tx.nearXidVersion() + - ", node=" + nodeId + - ", res=" + res + - ", fut=" + this + ']'); - } + if (!found && msgLog.isDebugEnabled()) { + msgLog.debug("Near finish fut, failed to find mini future [txId=" + tx.nearXidVersion() + + ", node=" + nodeId + + ", res=" + res + + ", fut=" + this + ']'); + } + }); } else { if (msgLog.isDebugEnabled()) { @@ -409,14 +411,16 @@ public void finish(final boolean commit, final boolean clearThreadMap, final boo if (tx.onNeedCheckBackup()) { assert tx.onePhaseCommit(); + log.warning("One-phase commit -> " + cctx.tm().finisher().order(tx) + " " + tx); + checkBackup(); + cctx.tm().finisher().finishSend(tx); + // If checkBackup is set, it means that primary node has crashed and we will not need to send // finish request to it, so we can mark future as initialized. markInitialized(); - log.warning("One-phase commit -> " + tx); - return; } @@ -635,11 +639,9 @@ private void checkBackup() { IgniteInternalFuture fut = cctx.tm().remoteTxFinishFuture(nearXidVer); - fut.listen(new CI1>() { - @Override public void apply(IgniteInternalFuture fut) { - mini.onDone(tx); - } - }); + fut.listen(f -> cctx.tm().finisher().execute(tx, () -> { + mini.onDone(tx); + })); return; } @@ -700,8 +702,6 @@ private void checkBackup() { else readyNearMappingFromBackup(mapping); } - - cctx.tm().finisher().finishSend(tx); } /** @@ -868,9 +868,6 @@ private void finish(int miniId, GridDistributedTxMapping m, boolean commit, bool } }); } - - if (miniId == mappings.mappings().size()) - cctx.tm().finisher().finishSend(tx); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java index 2b2fad927a341..ca1726970a302 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java @@ -49,8 +49,8 @@ public void execute(IgniteTxAdapter tx, Runnable transactionOp) { ordered++; } - //if (isNew) - //log.warning("New tx = " + (ordered - 1) + " " + tx); + if (isNew) + log.warning("New tx = " + (ordered - 1) + " " + tx); } transactionOp.run(); @@ -62,6 +62,8 @@ public void send(IgniteTxAdapter tx, Runnable transactionSendOp) { long order = txOrdering.get(txId); + log.warning("Send invoked -> " + order); + //log.warning("Try send " + order + " " + tx + ". Already sent: " + send); // Allowed to immediately send tx. @@ -82,6 +84,8 @@ public void finishSend(IgniteTxAdapter tx) { long order = txOrdering.get(txId); + log.warning("Finished send invoked -> " + order); + if (order >= send) if (!delayedSendings.containsKey(order)) delayedSendings.put(order, new ArrayList<>()); From 22a064fcf8ebf8b459cddecca5cc415727d882e2 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Tue, 21 Aug 2018 18:41:30 +0300 Subject: [PATCH 06/18] IGNITE-9270 Increased timeouts. --- .../processors/cache/transactions/IgniteTxFinisher.java | 2 +- .../eviction/paged/PageEvictionMultinodeAbstractTest.java | 4 +++- .../cache/transactions/TxRollbackAsyncTest.java | 8 +++++++- 3 files changed, 11 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java index ca1726970a302..41457456df6dc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java @@ -50,7 +50,7 @@ public void execute(IgniteTxAdapter tx, Runnable transactionOp) { } if (isNew) - log.warning("New tx = " + (ordered - 1) + " " + tx); + log.warning("New tx = " + (ordered - 1)); } transactionOp.run(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionMultinodeAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionMultinodeAbstractTest.java index 777c2d7e999cc..5c16eb05baef9 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionMultinodeAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionMultinodeAbstractTest.java @@ -64,7 +64,7 @@ public abstract class PageEvictionMultinodeAbstractTest extends PageEvictionAbst /** {@inheritDoc} */ @Override protected long getTestTimeout() { - return 10 * 60 * 1000; + return 20 * 60 * 1000; } /** @@ -85,6 +85,8 @@ public void testPageEviction() throws Exception { } } + + /** * @param cfg Config. * @throws Exception If failed. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackAsyncTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackAsyncTest.java index 4626dcd408eab..afeeac39c776c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackAsyncTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackAsyncTest.java @@ -138,6 +138,10 @@ public class TxRollbackAsyncTest extends GridCommonAbstractTest { return cfg; } + @Override protected long getTestTimeout() { + return 600 * 1000L; + } + /** * @return Near cache flag. */ @@ -430,6 +434,8 @@ private void testSynchronousRollback0(Ignite holdLockNode, final Ignite tryLockN assertEquals(0, holdLockNode.cache(CACHE_NAME).get(0)); + U.sleep(1000); + checkFutures(); } @@ -907,7 +913,7 @@ private void checkFutures() { for (GridCacheFuture fut : futs) log.info("Waiting for future: " + fut); - assertTrue("Expecting no active futures: node=" + ig.localNode().id(), futs.isEmpty()); + assertTrue("Expecting no active futures: node=" + ig.localNode().id() + " " + futs, futs.isEmpty()); } } From 10d9289f371a21539ea600d20a387efe7c8faa73 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 22 Aug 2018 14:00:44 +0300 Subject: [PATCH 07/18] IGNITE-9270 Shrink logging --- .../cache/distributed/dht/GridDhtTxLocalAdapter.java | 2 ++ .../cache/distributed/near/GridNearTxFinishFuture.java | 2 ++ .../cache/distributed/near/GridNearTxLocal.java | 2 ++ .../processors/cache/transactions/IgniteTxFinisher.java | 8 +++++--- 4 files changed, 11 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java index c5ed8849efc38..837fca67d2015 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java @@ -788,7 +788,9 @@ private IgniteInternalFuture obtainLockAsync( } } +/* log.warning("Finish thread id = " + Thread.currentThread().getName()); +*/ if (!Thread.currentThread().getName().contains("dedicated")) { throw new AssertionError("Commit requested not from dedicated stipe"); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java index 24cb2565b6d0a..f553a564393a8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java @@ -411,7 +411,9 @@ public void finish(final boolean commit, final boolean clearThreadMap, final boo if (tx.onNeedCheckBackup()) { assert tx.onePhaseCommit(); +/* log.warning("One-phase commit -> " + cctx.tm().finisher().order(tx) + " " + tx); +*/ checkBackup(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java index 2eedb6db6c09c..4810f0f693d2b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java @@ -3182,7 +3182,9 @@ private void readyNearLock(IgniteTxEntry txEntry, return new IgniteTxCommitFuture(t); } +/* log.warning("Finish thread id = " + Thread.currentThread().getName()); +*/ if (!Thread.currentThread().getName().contains("dedicated")) { throw new AssertionError("Commit requested not from dedicated stipe"); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java index 41457456df6dc..cb950523f2647 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java @@ -49,8 +49,10 @@ public void execute(IgniteTxAdapter tx, Runnable transactionOp) { ordered++; } +/* if (isNew) log.warning("New tx = " + (ordered - 1)); +*/ } transactionOp.run(); @@ -62,7 +64,7 @@ public void send(IgniteTxAdapter tx, Runnable transactionSendOp) { long order = txOrdering.get(txId); - log.warning("Send invoked -> " + order); + //log.warning("Send invoked -> " + order); //log.warning("Try send " + order + " " + tx + ". Already sent: " + send); @@ -84,7 +86,7 @@ public void finishSend(IgniteTxAdapter tx) { long order = txOrdering.get(txId); - log.warning("Finished send invoked -> " + order); + //log.warning("Finished send invoked -> " + order); if (order >= send) if (!delayedSendings.containsKey(order)) @@ -101,7 +103,7 @@ public void finishSend(IgniteTxAdapter tx) { //log.warning("Send incremented: " + send); while (delayedSendings.containsKey(send)) { - log.warning("Send delayed 2 - > " + send); + //log.warning("Send delayed 2 - > " + send); List delayed = delayedSendings.remove(send); txs.remove(send); From 3a39b1d4d824a51cdb1768b4fd6720d0b68ee85a Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 22 Aug 2018 14:02:37 +0300 Subject: [PATCH 08/18] IGNITE-9270 Shrink logging --- .../cache/distributed/near/GridNearTxFinishFuture.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java index f553a564393a8..3315e632882de 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java @@ -478,7 +478,9 @@ private void tryRollbackAsync(boolean onTimeout) { private void doFinish(boolean commit, boolean clearThreadMap) { IgniteTxCommitFuture commitFuture = tx.startLocalCommit(commit, clearThreadMap); +/* log.warning("Local commit started " + cctx.tm().finisher().order(tx) + " " + commitFuture + " " + commitFuture.started() + " " + commitFuture.async() + " " + commitFuture.commitEntriesFuture()); +*/ if (commitFuture.async()) commitFuture.listen(f -> cctx.tm().finisher().execute(tx, () -> doFinishAsync(commitFuture, commit, clearThreadMap))); @@ -497,7 +499,9 @@ private void doFinishAsync(IgniteTxCommitFuture commitFuture, boolean commit, bo if (commitFuture.started()) { tx.finishLocalCommit(commitFuture, commit, clearThreadMap); +/* log.warning("Local commit finished " + cctx.tm().finisher().order(tx) + " " + commitFuture + " " + commitFuture.started() + " " + commitFuture.async() + " " + commitFuture.commitEntriesFuture()); +*/ } else commitFuture.get(); // Check for errors. From 4cce9044539fdad176e53ec1cb2a72db8bd67649 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 22 Aug 2018 15:21:16 +0300 Subject: [PATCH 09/18] IGNITE-9270 Fixed tx finish. --- .../distributed/near/GridNearTxLocal.java | 53 +++++++++---------- 1 file changed, 26 insertions(+), 27 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java index 4810f0f693d2b..1d4bf3ed6aff4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java @@ -3434,40 +3434,39 @@ private IgniteInternalFuture chainFinishFuture(final NearTxFin if (!commit) { final GridNearTxFinishFuture rollbackFut = new GridNearTxFinishFuture<>(cctx, this, false); - fut.listen(new IgniteInClosure>() { - @Override public void apply(IgniteInternalFuture fut0) { - if (FINISH_FUT_UPD.compareAndSet(tx, fut, rollbackFut)) { - if (tx.state() == COMMITTED) { - if (log.isDebugEnabled()) - log.debug("Failed to rollback, transaction is already committed: " + tx); - - rollbackFut.forceFinish(); + fut.listen(f -> cctx.tm().finisher().execute(tx, () -> { + if (FINISH_FUT_UPD.compareAndSet(tx, fut, rollbackFut)) { + if (tx.state() == COMMITTED) { + if (log.isDebugEnabled()) + log.debug("Failed to rollback, transaction is already committed: " + tx); - assert rollbackFut.isDone() : rollbackFut; - } - else { - if (!cctx.mvcc().addFuture(rollbackFut, rollbackFut.futureId())) - return; + rollbackFut.forceFinish(); - rollbackFut.finish(false, clearThreadMap, onTimeout); - } + assert rollbackFut.isDone() : rollbackFut; } else { - finishFut.listen(new IgniteInClosure>() { - @Override public void apply(IgniteInternalFuture fut) { - try { - fut.get(); + if (!cctx.mvcc().addFuture(rollbackFut, rollbackFut.futureId())) + return; - rollbackFut.markInitialized(); - } - catch (IgniteCheckedException e) { - rollbackFut.onDone(e); - } - } - }); + rollbackFut.finish(false, clearThreadMap, onTimeout); } } - }); + else { + finishFut.listen(new IgniteInClosure>() { + @Override public void apply(IgniteInternalFuture fut) { + try { + fut.get(); + + rollbackFut.markInitialized(); + } + catch (IgniteCheckedException e) { + rollbackFut.onDone(e); + } + } + }); + } + }) + ); return rollbackFut; } From 32f385407d5b276f622a577cae97bf54351bebef Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 22 Aug 2018 18:57:44 +0300 Subject: [PATCH 10/18] IGNITE-9270 WIP. --- .../GridDistributedTxRemoteAdapter.java | 885 ++++++++++-------- .../cache/transactions/IgniteTxFinisher.java | 10 +- .../cache/transactions/IgniteTxHandler.java | 49 +- .../cache/transactions/IgniteTxRemoteEx.java | 9 +- 4 files changed, 557 insertions(+), 396 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java index 1b9b3a8ad28fd..bcd5d41320b56 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java @@ -18,7 +18,6 @@ package org.apache.ignite.internal.processors.cache.distributed; import java.io.Externalizable; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.LinkedList; @@ -27,15 +26,13 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; -import java.util.stream.Collectors; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.InvalidEnvironmentException; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.InvalidEnvironmentException; import org.apache.ignite.internal.pagemem.wal.StorageException; import org.apache.ignite.internal.pagemem.wal.WALPointer; -import org.apache.ignite.internal.pagemem.wal.record.DataEntry; -import org.apache.ignite.internal.pagemem.wal.record.DataRecord; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheEntryOperationFuture; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; @@ -51,6 +48,8 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheEntry; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxCommitEntriesFuture; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxCommitFuture; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxRemoteEx; @@ -60,11 +59,11 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext; import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException; import org.apache.ignite.internal.util.future.GridFinishedFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.lang.GridTuple; import org.apache.ignite.internal.util.tostring.GridToStringBuilder; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; @@ -332,21 +331,23 @@ private void doneRemote(IgniteTxEntry txEntry, if (!hasWriteKey(entry.txKey())) return false; - try { - commitIfLocked(); + IgniteTxCommitFuture commitFuture = startCommit(); - return true; - } - catch (IgniteCheckedException e) { - U.error(log, "Failed to commit remote transaction: " + this, e); + commitFuture.listen(f -> cctx.tm().finisher().execute(this, () -> { + try { + finishCommit(commitFuture); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to commit remote transaction: " + this, e); - invalidate(true); - systemInvalidate(true); + invalidate(true); + systemInvalidate(true); - rollbackRemoteTx(); + rollbackRemoteTx(); + } + })); - return false; - } + return true; } /** {@inheritDoc} */ @@ -416,419 +417,428 @@ public final void prepareRemoteTx() throws IgniteCheckedException { * @throws IgniteCheckedException If commit failed. */ @SuppressWarnings({"CatchGenericClass"}) - private void commitIfLocked() throws IgniteCheckedException { - if (state() == COMMITTING) { - for (IgniteTxEntry txEntry : writeEntries()) { - assert txEntry != null : "Missing transaction entry for tx: " + this; + private IgniteTxCommitEntriesFuture startCommitEntries() { + if (state() != COMMITTING) + return IgniteTxCommitEntriesFuture.FINISHED; + + if (checkLocks()) + return IgniteTxCommitEntriesFuture.FINISHED; + + Map writeMap = txState.writeMap(); - while (true) { - GridCacheEntryEx entry = txEntry.cached(); + if (F.isEmpty(writeMap)) + return IgniteTxCommitEntriesFuture.FINISHED; - assert entry != null : "Missing cached entry for transaction entry: " + txEntry; + IgniteCheckedException err = null; - try { - GridCacheVersion ver = txEntry.explicitVersion() != null ? txEntry.explicitVersion() : xidVer; + GridCacheReturnCompletableWrapper wrapper = null; - // If locks haven't been acquired yet, keep waiting. - if (!entry.lockedBy(ver)) { - if (log.isDebugEnabled()) - log.debug("Transaction does not own lock for entry (will wait) [entry=" + entry + - ", tx=" + this + ']'); + GridCacheReturn ret = null; - return; - } + if (!near() && !local() && onePhaseCommit()) { + if (needReturnValue()) { + ret = new GridCacheReturn(null, cctx.localNodeId().equals(otherNodeId()), true, null, true); + + UUID origNodeId = otherNodeId(); // Originating node. + + cctx.tm().addCommittedTxReturn(this, + wrapper = new GridCacheReturnCompletableWrapper( + !cctx.localNodeId().equals(origNodeId) ? origNodeId : null)); + } + else + cctx.tm().addCommittedTx(this, this.nearXidVersion(), null); + } - break; // While. + // Register this transaction as completed prior to write-phase to + // ensure proper lock ordering for removed entries. + cctx.tm().addCommittedTx(this); + + AffinityTopologyVersion topVer = topologyVersion(); + + try { + IgniteTxCommitEntriesFuture commitEntriesFut = new IgniteTxCommitEntriesFuture(); + + Collection entries = commitEntries(); + + try { + batchStoreCommit(writeMap().values()); + + // Node that for near transactions we grab all entries. + for (IgniteTxEntry txEntry : entries) { + // Prepare context for transaction entry. + TransactionEntryContext txContext = prepareContext(txEntry); + + // Nothing to perform. + if (txContext == null) { + commitEntriesFut.add(new GridFinishedFuture<>()); + + continue; } - catch (GridCacheEntryRemovedException ignore) { + + CacheEntryOperationFuture operationFut; + + if (txContext.operation == CREATE || txContext.operation == UPDATE) { + operationFut = cctx.cache().executor().execute(txContext.entry, e -> { + // Invalidate only for near nodes (backups cannot be invalidated). + if (isSystemInvalidate() || (isInvalidate() && txContext.cacheContext.isNear())) + return e.innerRemove(this, + eventNodeId(), + nodeId, + false, + true, + true, + txEntry.keepBinary(), + txEntry.hasOldValue(), + txEntry.oldValue(), + topVer, + null, + txContext.cacheContext.isDrEnabled() ? DR_BACKUP : DR_NONE, + near() ? null : txContext.explicitVer, + CU.subjectId(this, cctx), + resolveTaskName(), + txContext.dhtVer, + null); + else { + assert txContext.value != null : txEntry; + + return e.innerSet(this, + eventNodeId(), + nodeId, + txContext.value, + false, + false, + txEntry.ttl(), + true, + true, + txEntry.keepBinary(), + txEntry.hasOldValue(), + txEntry.oldValue(), + topVer, + null, + txContext.cacheContext.isDrEnabled() ? DR_BACKUP : DR_NONE, + txEntry.conflictExpireTime(), + near() ? null : txContext.explicitVer, + CU.subjectId(this, cctx), + resolveTaskName(), + txContext.dhtVer, + null); + } + }, + e -> refresh(txEntry), + (e, result) -> { + if (txContext.nearEntry == null) + return; + + // Keep near entry up to date. + try { + CacheObject valBytes = e.valueBytes(); + + txContext.nearEntry.updateOrEvict(xidVer, + valBytes, + e.expireTime(), + e.ttl(), + nodeId, + topVer); + } catch (GridCacheEntryRemovedException re) { + // This is impossible in TPP way, but anyway throw exception. + throw new IgniteCheckedException("Failed to get value bytes for entry: " + e, re); + } + }); + } else if (txContext.operation == DELETE) { + operationFut = cctx.cache().executor().execute(txContext.entry, e -> { + return e.innerRemove(this, + eventNodeId(), + nodeId, + false, + true, + true, + txEntry.keepBinary(), + txEntry.hasOldValue(), + txEntry.oldValue(), + topVer, + null, + txContext.cacheContext.isDrEnabled() ? DR_BACKUP : DR_NONE, + near() ? null : txContext.explicitVer, + CU.subjectId(this, cctx), + resolveTaskName(), + txContext.dhtVer, + txEntry.updateCounter() + ); + }, + e -> refresh(txEntry), + (e, result) -> { + // Keep near entry up to date. + if (txContext.nearEntry != null) + txContext.nearEntry.updateOrEvict(xidVer, null, 0, 0, nodeId, topVer); + }); + } else if (txContext.operation == RELOAD) { + operationFut = cctx.cache().executor().execute(txContext.entry, e -> { + CacheObject val = e.innerReload(); + + return new GridCacheUpdateTxResult(true, val, null); + }, + e -> refresh(txEntry), + (e, result) -> { + if (txContext.nearEntry == null) + return; + + // Keep near entry up to date. + GridNearCacheEntry nearEntry = txContext.nearEntry; + + for (; ; ) { + try { + CacheObject reloaded = nearEntry.innerReload(); + + nearEntry.updateOrEvict(e.version(), + reloaded, + e.expireTime(), + e.ttl(), + nodeId, + topVer); + + break; + } catch (GridCacheEntryRemovedException re) { + nearEntry = txContext.cacheContext.dht().near().peekExx(txEntry.key()); + } + } + } + ); + } else if (txContext.operation == READ) { + assert near(); + + operationFut = new CacheEntryOperationFuture<>(); + + operationFut.onDone(new GridCacheUpdateTxResult(false, null, null)); + if (log.isDebugEnabled()) - log.debug("Got removed entry while committing (will retry): " + txEntry); + log.debug("Ignoring READ entry when committing: " + txEntry); + } + // No-op. + else { + operationFut = cctx.cache().executor().execute(txContext.entry, e -> { + if (txContext.conflictContext == null || !txContext.conflictContext.isUseOld()) { + if (txEntry.ttl() != CU.TTL_NOT_CHANGED) + e.updateTtl(null, txEntry.ttl()); + } - txEntry.cached(txEntry.context().cache().entryEx(txEntry.key(), topologyVersion())); + return new GridCacheUpdateTxResult(true, null, null); + }, + e -> refresh(txEntry), + (e, result) -> { + if (txContext.nearEntry == null) + return; + + try { + CacheObject valBytes = e.valueBytes(); + + txContext.nearEntry.updateOrEvict(xidVer, + valBytes, + e.expireTime(), + e.ttl(), + nodeId, + topVer); + } catch (GridCacheEntryRemovedException re) { + // This is impossible in TPP way, but anyway throw exception. + throw new IgniteCheckedException("Failed to get value bytes for entry: " + e, re); + } + }); } + + assert operationFut != null; + + commitEntriesFut.add(operationFut); + + // Assert after setting values as we want to make sure + // that if we replaced removed entries. + assert + txEntry.op() == READ || onePhaseCommit() || + // If candidate is not there, then lock was explicit + // and we simply allow the commit to proceed. + !txContext.entry.hasLockCandidateUnsafe(xidVer) || txContext.entry.lockedByUnsafe(xidVer) : + "Transaction does not own lock for commit [entry=" + txContext.entry + + ", tx=" + this + ']'; } } + catch (Throwable t) { + commitEntriesFut.onDone(t); - // Only one thread gets to commit. - if (COMMIT_ALLOWED_UPD.compareAndSet(this, 0, 1)) { - IgniteCheckedException err = null; + return commitEntriesFut; + } - Map writeMap = txState.writeMap(); + commitEntriesFut.markInitialized(); - GridCacheReturnCompletableWrapper wrapper = null; + return commitEntriesFut; + } + finally { + if (wrapper != null) + wrapper.initialize(ret); + } + } - if (!F.isEmpty(writeMap)) { - GridCacheReturn ret = null; + private boolean checkLocks() { + for (IgniteTxEntry txEntry : writeEntries()) { + assert txEntry != null : "Missing transaction entry for tx: " + this; - if (!near() && !local() && onePhaseCommit()) { - if (needReturnValue()) { - ret = new GridCacheReturn(null, cctx.localNodeId().equals(otherNodeId()), true, null, true); + for (;;) { + GridCacheEntryEx entry = txEntry.cached(); - UUID origNodeId = otherNodeId(); // Originating node. + assert entry != null : "Missing cached entry for transaction entry: " + txEntry; - cctx.tm().addCommittedTxReturn(this, - wrapper = new GridCacheReturnCompletableWrapper( - !cctx.localNodeId().equals(origNodeId) ? origNodeId : null)); - } - else - cctx.tm().addCommittedTx(this, this.nearXidVersion(), null); - } + try { + GridCacheVersion ver = txEntry.explicitVersion() != null ? txEntry.explicitVersion() : xidVer; - // Register this transaction as completed prior to write-phase to - // ensure proper lock ordering for removed entries. - cctx.tm().addCommittedTx(this); + // If locks haven't been acquired yet, keep waiting. + if (!entry.lockedBy(ver)) { + if (log.isDebugEnabled()) + log.debug("Transaction does not own lock for entry (will wait) [entry=" + entry + + ", tx=" + this + ']'); - AffinityTopologyVersion topVer = topologyVersion(); + return true; + } - WALPointer ptr = null; + break; // While. + } + catch (GridCacheEntryRemovedException ignore) { + if (log.isDebugEnabled()) + log.debug("Got removed entry while committing (will retry): " + txEntry); - cctx.database().checkpointReadLock(); + txEntry.cached(txEntry.context().cache().entryEx(txEntry.key(), topologyVersion())); + } + } + } - try { - Collection entries = near() || cctx.snapshot().needTxReadLogging() ? allEntries() : writeEntries(); + return false; + } - // Data entry to write to WAL and associated with it TxEntry. - List> dataEntries = null; + private Collection commitEntries() { + return near() || cctx.snapshot().needTxReadLogging() ? allEntries() : writeEntries(); + } - batchStoreCommit(writeMap().values()); + public void finishCommit(IgniteTxCommitEntriesFuture commitEntriesFuture) throws IgniteCheckedException { + assert commitEntriesFuture.isDone(); - try { - // Node that for near transactions we grab all entries. - for (IgniteTxEntry txEntry : entries) { - GridCacheContext cacheCtx = txEntry.context(); + // Nothing to commit. + if (!commitEntriesFuture.initialized()) + return; - boolean replicate = cacheCtx.isDrEnabled(); + IgniteCheckedException err = null; - try { - while (true) { - try { - GridCacheEntryEx cached = txEntry.cached(); - - if (cached == null) - txEntry.cached(cached = cacheCtx.cache().entryEx(txEntry.key(), topologyVersion())); - - if (near() && cacheCtx.dr().receiveEnabled()) { - cached.markObsolete(xidVer); - - break; - } - - GridNearCacheEntry nearCached = null; - - if (updateNearCache(cacheCtx, txEntry.key(), topVer)) - nearCached = cacheCtx.dht().near().peekExx(txEntry.key()); - - if (!F.isEmpty(txEntry.entryProcessors())) - txEntry.cached().unswap(false); - - IgniteBiTuple res = - applyTransformClosures(txEntry, false, ret); - - GridCacheOperation op = res.get1(); - CacheObject val = res.get2(); - - GridCacheVersion explicitVer = txEntry.conflictVersion(); - - if (explicitVer == null) - explicitVer = writeVersion(); - - if (txEntry.ttl() == CU.TTL_ZERO) - op = DELETE; - - boolean conflictNeedResolve = cacheCtx.conflictNeedResolve(); - - GridCacheVersionConflictContext conflictCtx = null; - - if (conflictNeedResolve) { - IgniteBiTuple - drRes = conflictResolve(op, txEntry, val, explicitVer, cached); - - assert drRes != null; - - conflictCtx = drRes.get2(); - - if (conflictCtx.isUseOld()) - op = NOOP; - else if (conflictCtx.isUseNew()) { - txEntry.ttl(conflictCtx.ttl()); - txEntry.conflictExpireTime(conflictCtx.expireTime()); - } - else if (conflictCtx.isMerge()) { - op = drRes.get1(); - val = txEntry.context().toCacheObject(conflictCtx.mergeValue()); - explicitVer = writeVersion(); - - txEntry.ttl(conflictCtx.ttl()); - txEntry.conflictExpireTime(conflictCtx.expireTime()); - } - } - else - // Nullify explicit version so that innerSet/innerRemove will work as usual. - explicitVer = null; - - GridCacheVersion dhtVer = cached.isNear() ? writeVersion() : null; - - if (!near() && cacheCtx.group().persistenceEnabled() && cacheCtx.group().walEnabled() && - op != NOOP && op != RELOAD && (op != READ || cctx.snapshot().needTxReadLogging())) { - if (dataEntries == null) - dataEntries = new ArrayList<>(entries.size()); - - dataEntries.add( - new T2<>( - new DataEntry( - cacheCtx.cacheId(), - txEntry.key(), - val, - op, - nearXidVersion(), - writeVersion(), - 0, - txEntry.key().partition(), - txEntry.updateCounter() - ), - txEntry - ) - ); - } - - if (op == CREATE || op == UPDATE) { - // Invalidate only for near nodes (backups cannot be invalidated). - if (isSystemInvalidate() || (isInvalidate() && cacheCtx.isNear())) - cached.innerRemove(this, - eventNodeId(), - nodeId, - false, - true, - true, - txEntry.keepBinary(), - txEntry.hasOldValue(), - txEntry.oldValue(), - topVer, - null, - replicate ? DR_BACKUP : DR_NONE, - near() ? null : explicitVer, - CU.subjectId(this, cctx), - resolveTaskName(), - dhtVer, - txEntry.updateCounter()); - else { - assert val != null : txEntry; - - GridCacheUpdateTxResult updRes = cached.innerSet(this, - eventNodeId(), - nodeId, - val, - false, - false, - txEntry.ttl(), - true, - true, - txEntry.keepBinary(), - txEntry.hasOldValue(), - txEntry.oldValue(), - topVer, - null, - replicate ? DR_BACKUP : DR_NONE, - txEntry.conflictExpireTime(), - near() ? null : explicitVer, - CU.subjectId(this, cctx), - resolveTaskName(), - dhtVer, - txEntry.updateCounter()); - - txEntry.updateCounter(updRes.updatePartitionCounter()); - - if (updRes.loggedPointer() != null) - ptr = updRes.loggedPointer(); - - // Keep near entry up to date. - if (nearCached != null) { - CacheObject val0 = cached.valueBytes(); - - nearCached.updateOrEvict(xidVer, - val0, - cached.expireTime(), - cached.ttl(), - nodeId, - topVer); - } - } - } - else if (op == DELETE) { - GridCacheUpdateTxResult updRes = cached.innerRemove(this, - eventNodeId(), - nodeId, - false, - true, - true, - txEntry.keepBinary(), - txEntry.hasOldValue(), - txEntry.oldValue(), - topVer, - null, - replicate ? DR_BACKUP : DR_NONE, - near() ? null : explicitVer, - CU.subjectId(this, cctx), - resolveTaskName(), - dhtVer, - txEntry.updateCounter()); - - txEntry.updateCounter(updRes.updatePartitionCounter()); - - if (updRes.loggedPointer() != null) - ptr = updRes.loggedPointer(); - - // Keep near entry up to date. - if (nearCached != null) - nearCached.updateOrEvict(xidVer, null, 0, 0, nodeId, topVer); - } - else if (op == RELOAD) { - CacheObject reloaded = cached.innerReload(); - - if (nearCached != null) { - nearCached.innerReload(); - - nearCached.updateOrEvict(cached.version(), - reloaded, - cached.expireTime(), - cached.ttl(), - nodeId, - topVer); - } - } - else if (op == READ) { - assert near(); - - if (log.isDebugEnabled()) - log.debug("Ignoring READ entry when committing: " + txEntry); - } - // No-op. - else { - if (conflictCtx == null || !conflictCtx.isUseOld()) { - if (txEntry.ttl() != CU.TTL_NOT_CHANGED) - cached.updateTtl(null, txEntry.ttl()); - - if (nearCached != null) { - CacheObject val0 = cached.valueBytes(); - - nearCached.updateOrEvict(xidVer, - val0, - cached.expireTime(), - cached.ttl(), - nodeId, - topVer); - } - } - } - - // Assert after setting values as we want to make sure - // that if we replaced removed entries. - assert - txEntry.op() == READ || onePhaseCommit() || - // If candidate is not there, then lock was explicit - // and we simply allow the commit to proceed. - !cached.hasLockCandidateUnsafe(xidVer) || cached.lockedByUnsafe(xidVer) : - "Transaction does not own lock for commit [entry=" + cached + - ", tx=" + this + ']'; - - // Break out of while loop. - break; - } - catch (GridCacheEntryRemovedException ignored) { - if (log.isDebugEnabled()) - log.debug("Attempting to commit a removed entry (will retry): " + txEntry); - - // Renew cached entry. - txEntry.cached(cacheCtx.cache().entryEx(txEntry.key(), topologyVersion())); - } - } - } - catch (Throwable ex) { - boolean hasIOIssue = X.hasCause(ex, InvalidEnvironmentException.class); + try { + WALPointer latestPtr = null; - // In case of error, we still make the best effort to commit, - // as there is no way to rollback at this point. - err = new IgniteTxHeuristicCheckedException("Commit produced a runtime exception " + - "(all transaction entries will be invalidated): " + CU.txString(this), ex); + try { + commitEntriesFuture.get(); // Check for errors. - if (hasIOIssue) { - U.warn(log, "Failed to commit transaction, node is stopping [tx=" + this + - ", err=" + ex + ']'); - } - else - U.error(log, "Commit failed.", err); + Collection commitEntries = commitEntries(); - uncommit(hasIOIssue); + List> futures = commitEntriesFuture.futures(); - state(UNKNOWN); + assert commitEntries.size() == futures.size(); - if (ex instanceof Error) - throw (Error)ex; - } - } + int txEntryIdx = 0; - if (!near() && !F.isEmpty(dataEntries) && cctx.wal() != null) { - // Set new update counters for data entries received from persisted tx entries. - List entriesWithCounters = dataEntries.stream() - .map(tuple -> tuple.get1().partitionCounter(tuple.get2().updateCounter())) - .collect(Collectors.toList()); + for (IgniteTxEntry txEntry : commitEntries) { + GridCacheUpdateTxResult txResult = futures.get(txEntryIdx).get(); - cctx.wal().log(new DataRecord(entriesWithCounters)); - } + if (txResult != null && txResult.success()) { + if (txResult.updatePartitionCounter() != -1) + txEntry.updateCounter(txResult.updatePartitionCounter()); - if (ptr != null && !cctx.tm().logTxRecords()) - cctx.wal().flush(ptr, false); - } - catch (StorageException e) { - throw new IgniteCheckedException("Failed to log transaction record " + - "(transaction will be rolled back): " + this, e); - } - } - finally { - cctx.database().checkpointReadUnlock(); + WALPointer loggedPtr = txResult.loggedPointer(); - if (wrapper != null) - wrapper.initialize(ret); + // Find latest logged WAL pointer. + if (loggedPtr != null) + if (latestPtr == null || loggedPtr.compareTo(latestPtr) > 0) + latestPtr = loggedPtr; } + + txEntryIdx++; } + } + catch (Throwable ex) { + boolean hasIOIssue = X.hasCause(ex, InvalidEnvironmentException.class); - if (err != null) { - state(UNKNOWN); + // In case of error, we still make the best effort to commit, + // as there is no way to rollback at this point. + err = new IgniteTxHeuristicCheckedException("Commit produced a runtime exception " + + "(all transaction entries will be invalidated): " + CU.txString(this), ex); - throw err; + if (hasIOIssue) { + U.warn(log, "Failed to commit transaction, node is stopping [tx=" + this + + ", err=" + ex + ']'); } + else + U.error(log, "Commit failed.", err); + + uncommit(hasIOIssue); - cctx.tm().commitTx(this); + state(UNKNOWN); + + if (ex instanceof Error) + throw (Error)ex; + } - state(COMMITTED); + if (err != null) { + if (latestPtr != null && !cctx.tm().logTxRecords()) + cctx.wal().flush(latestPtr, false); } } + catch (StorageException e) { + throw new IgniteCheckedException("Failed to log transaction record " + + "(transaction will be rolled back): " + this, e); + } + + if (err != null) { + state(UNKNOWN); + + throw err; + } + + cctx.tm().commitTx(this); + + state(COMMITTED); } /** {@inheritDoc} */ - @Override public final void commitRemoteTx() throws IgniteCheckedException { - if (optimistic()) - state(PREPARED); + @Override public IgniteTxCommitFuture startCommit() { + try { + if (optimistic()) + state(PREPARED); - if (!state(COMMITTING)) { - TransactionState state = state(); + if (!state(COMMITTING)) { + TransactionState state = state(); - // If other thread is doing commit, then no-op. - if (state == COMMITTING || state == COMMITTED) - return; + // If other thread is doing commit, then no-op. + if (state == COMMITTING || state == COMMITTED) + return new IgniteTxCommitFuture(false); + + if (log.isDebugEnabled()) + log.debug("Failed to set COMMITTING transaction state (will rollback): " + this); - if (log.isDebugEnabled()) - log.debug("Failed to set COMMITTING transaction state (will rollback): " + this); + setRollbackOnly(); - setRollbackOnly(); + if (!isSystemInvalidate()) + throw new IgniteCheckedException("Invalid transaction state for commit [state=" + state + ", tx=" + this + ']'); - if (!isSystemInvalidate()) - throw new IgniteCheckedException("Invalid transaction state for commit [state=" + state + ", tx=" + this + ']'); + rollbackRemoteTx(); - rollbackRemoteTx(); + return new IgniteTxCommitFuture(true); + } + + return new IgniteTxCommitFuture(startCommitEntries(), true); + } + catch (Throwable t) { + return new IgniteTxCommitFuture(t); } + } + + @Override public void finishCommit(IgniteTxCommitFuture commitFuture) throws IgniteCheckedException { + assert commitFuture.isDone(); - commitIfLocked(); + commitFuture.get(); // Check for errors. + + if (commitFuture.commitEntriesFuture() != null) + finishCommit(commitFuture.commitEntriesFuture()); } /** @@ -837,19 +847,27 @@ else if (op == READ) { * @throws IgniteCheckedException If commit failed. */ public void forceCommit() throws IgniteCheckedException { - commitIfLocked(); + commitAsync(); } /** {@inheritDoc} */ @Override public IgniteInternalFuture commitAsync() { - try { - commitRemoteTx(); + GridFutureAdapter fut = new GridFutureAdapter<>(); - return new GridFinishedFuture(this); - } - catch (IgniteCheckedException e) { - return new GridFinishedFuture<>(e); - } + IgniteTxCommitFuture commitFut = startCommit(); + + commitFut.listen(f -> cctx.tm().finisher().execute(this, () -> { + try { + finishCommit(commitFut); + + fut.onDone(this); + } + catch (IgniteCheckedException e) { + fut.onDone(e); + } + })); + + return fut; } /** {@inheritDoc} */ @@ -872,7 +890,7 @@ public void forceCommit() throws IgniteCheckedException { Collections.emptyList(), Collections.emptyList()); - commitRemoteTx(); + return commitAsync(); } catch (IgniteCheckedException e) { U.error(log, "Failed to invalidate transaction: " + xidVersion(), e); @@ -958,4 +976,109 @@ protected void addExplicit(IgniteTxEntry e) { return GridToStringBuilder.toString(GridDistributedTxRemoteAdapter.class, this, "super", super.toString()); } + private GridCacheEntryEx refresh(IgniteTxEntry txEntry) { + return txEntry.context().cache().entryEx(txEntry.key(), topologyVersion()); + } + + private TransactionEntryContext prepareContext(IgniteTxEntry txEntry) throws IgniteCheckedException { + GridCacheContext cacheCtx = txEntry.context(); + + for (;;) { + GridCacheEntryEx cached = txEntry.cached(); + + try { + if (cached == null) + txEntry.cached(cached = cacheCtx.cache().entryEx(txEntry.key(), topologyVersion())); + + if (near() && cacheCtx.dr().receiveEnabled()) { + cached.markObsolete(xidVer); + + return null; + } + + final GridNearCacheEntry nearCached = updateNearCache(cacheCtx, txEntry.key(), topVer) + ? cacheCtx.dht().near().peekExx(txEntry.key()) + : null; + + if (!F.isEmpty(txEntry.entryProcessors())) + txEntry.cached().unswap(false); + + IgniteBiTuple res = applyTransformClosures(txEntry, false, null); + + GridCacheOperation op = res.get1(); + CacheObject val = res.get2(); + + GridCacheVersion explicitVer = txEntry.conflictVersion(); + + if (explicitVer == null) + explicitVer = writeVersion(); + + if (txEntry.ttl() == CU.TTL_ZERO) + op = DELETE; + + boolean conflictNeedResolve = cacheCtx.conflictNeedResolve(); + + GridCacheVersionConflictContext conflictCtx = null; + + if (conflictNeedResolve) { + IgniteBiTuple + drRes = conflictResolve(op, txEntry, val, explicitVer, cached); + + assert drRes != null; + + conflictCtx = drRes.get2(); + + if (conflictCtx.isUseOld()) + op = NOOP; + else if (conflictCtx.isUseNew()) { + txEntry.ttl(conflictCtx.ttl()); + txEntry.conflictExpireTime(conflictCtx.expireTime()); + } + else if (conflictCtx.isMerge()) { + op = drRes.get1(); + val = txEntry.context().toCacheObject(conflictCtx.mergeValue()); + explicitVer = writeVersion(); + + txEntry.ttl(conflictCtx.ttl()); + txEntry.conflictExpireTime(conflictCtx.expireTime()); + } + } + else + // Nullify explicit version so that innerSet/innerRemove will work as usual. + explicitVer = null; + + GridCacheVersion dhtVer = cached.isNear() ? writeVersion() : null; + + return new TransactionEntryContext(cacheCtx, op, dhtVer, explicitVer, val, cached, nearCached, conflictCtx); + } + catch (GridCacheEntryRemovedException re) { + txEntry.cached(cacheCtx.cache().entryEx(txEntry.key(), topologyVersion())); + } + } + } + + /** + * Context to perform operation with {@link IgniteTxEntry}. + */ + class TransactionEntryContext { + private final GridCacheContext cacheContext; + private final GridCacheOperation operation; + private final GridCacheVersion dhtVer; + private final GridCacheVersion explicitVer; + private final CacheObject value; + private final GridCacheEntryEx entry; + private final GridNearCacheEntry nearEntry; + private final GridCacheVersionConflictContext conflictContext; + + public TransactionEntryContext(GridCacheContext cacheContext, GridCacheOperation operation, GridCacheVersion dhtVer, GridCacheVersion explicitVer, CacheObject value, GridCacheEntryEx entry, GridNearCacheEntry nearEntry, GridCacheVersionConflictContext conflictContext) { + this.cacheContext = cacheContext; + this.operation = operation; + this.dhtVer = dhtVer; + this.explicitVer = explicitVer; + this.value = value; + this.entry = entry; + this.nearEntry = nearEntry; + this.conflictContext = conflictContext; + } + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java index cb950523f2647..0d3a0d52c0516 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java @@ -21,7 +21,7 @@ public class IgniteTxFinisher { private final ConcurrentLinkedHashMap txOrdering = new ConcurrentLinkedHashMap(); - private final ConcurrentLinkedHashMap txs = new ConcurrentLinkedHashMap<>(); + private final ConcurrentLinkedHashMap txs = new ConcurrentLinkedHashMap<>(); private final Map> delayedSendings = new LinkedHashMap<>(); @@ -34,7 +34,7 @@ public IgniteTxFinisher(GridCacheSharedContext cctx) { this.log = cctx.logger(getClass()); } - public void execute(IgniteTxAdapter tx, Runnable transactionOp) { + public void execute(IgniteInternalTx tx, Runnable transactionOp) { cctx.kernalContext().getStripedExecutorService().executeDedicated(DEDICATED_WORKER_IDX, () -> { GridCacheVersion txId = tx.xidVersion(); @@ -59,7 +59,7 @@ public void execute(IgniteTxAdapter tx, Runnable transactionOp) { }); } - public void send(IgniteTxAdapter tx, Runnable transactionSendOp) { + public void send(IgniteInternalTx tx, Runnable transactionSendOp) { GridCacheVersion txId = tx.xidVersion(); long order = txOrdering.get(txId); @@ -81,7 +81,7 @@ public void send(IgniteTxAdapter tx, Runnable transactionSendOp) { } } - public void finishSend(IgniteTxAdapter tx) { + public void finishSend(IgniteInternalTx tx) { GridCacheVersion txId = tx.xidVersion(); long order = txOrdering.get(txId); @@ -116,7 +116,7 @@ public void finishSend(IgniteTxAdapter tx) { } } - public long order(IgniteTxAdapter tx) { + public long order(IgniteInternalTx tx) { return txOrdering.get(tx.xidVersion()); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java index eb97efd03e386..a509a7c55aa7c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java @@ -1335,7 +1335,34 @@ else if (log.isDebugEnabled()) tx.setPartitionUpdateCounters( req.partUpdateCounters() != null ? req.partUpdateCounters().array() : null); - tx.commitRemoteTx(); + IgniteTxCommitFuture commitFut = tx.startCommit(); + + commitFut.listen(f -> ctx.tm().finisher().execute(tx, () -> { + try { + tx.finishCommit(commitFut); + } + catch (Throwable e) { + U.error(log, "Failed completing transaction [commit=" + req.commit() + ", tx=" + tx + ']', e); + + // Mark transaction for invalidate. + tx.invalidate(true); + tx.systemInvalidate(true); + + IgniteTxCommitFuture commitFuture = tx.startCommit(); + + commitFuture.listen(f0 -> ctx.tm().finisher().execute(tx, () -> { + try { + tx.finishCommit(commitFuture); + } + catch (IgniteCheckedException ex) { + U.error(log, "Failed to invalidate transaction: " + tx, ex); + } + })); + + if (e instanceof Error) + throw (Error)e; + } + })); } else { tx.doneRemote(req.baseVersion(), null, null, null); @@ -1350,12 +1377,16 @@ else if (log.isDebugEnabled()) tx.invalidate(true); tx.systemInvalidate(true); - try { - tx.commitRemoteTx(); - } - catch (IgniteCheckedException ex) { - U.error(log, "Failed to invalidate transaction: " + tx, ex); - } + IgniteTxCommitFuture commitFuture = tx.startCommit(); + + commitFuture.listen(f -> ctx.tm().finisher().execute(tx, () -> { + try { + tx.finishCommit(commitFuture); + } + catch (IgniteCheckedException ex) { + U.error(log, "Failed to invalidate transaction: " + tx, ex); + } + })); if (e instanceof Error) throw (Error)e; @@ -1366,7 +1397,7 @@ else if (log.isDebugEnabled()) * @param tx Transaction. * @param req Request. */ - protected void finish( + private void finish( GridDistributedTxRemoteAdapter tx, GridDhtTxPrepareRequest req) throws IgniteTxHeuristicCheckedException { assert tx != null : "No transaction for one-phase commit prepare request: " + req; @@ -1378,7 +1409,7 @@ protected void finish( // Complete remote candidates. tx.doneRemote(req.version(), null, null, null); - tx.commitRemoteTx(); + tx.startCommit(); } catch (IgniteTxHeuristicCheckedException e) { // Just rethrow this exception. Transaction was already uncommitted. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteEx.java index 87cc7ccea3635..17e1caa4903ff 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxRemoteEx.java @@ -25,10 +25,17 @@ * Local transaction API. */ public interface IgniteTxRemoteEx extends IgniteInternalTx { + /** + * Starts asynchronous commit. + * + * @return Commit future. + */ + public IgniteTxCommitFuture startCommit(); + /** * @throws IgniteCheckedException If failed. */ - public void commitRemoteTx() throws IgniteCheckedException; + public void finishCommit(IgniteTxCommitFuture commitFuture) throws IgniteCheckedException; /** * From 0ff9bb1fde5fc2f6c3d950ec98bab17a4ca2923d Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 22 Aug 2018 19:53:56 +0300 Subject: [PATCH 11/18] IGNITE-9270 WIP. --- .../GridDistributedTxRemoteAdapter.java | 60 +++--- .../dht/GridDhtTxLocalAdapter.java | 19 +- .../distributed/near/GridNearTxLocal.java | 15 +- .../transactions/IgniteTxCommitFuture.java | 2 + .../cache/transactions/IgniteTxHandler.java | 203 ++++++++++++------ 5 files changed, 184 insertions(+), 115 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java index bcd5d41320b56..bb99f76c9acbc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java @@ -331,21 +331,23 @@ private void doneRemote(IgniteTxEntry txEntry, if (!hasWriteKey(entry.txKey())) return false; - IgniteTxCommitFuture commitFuture = startCommit(); + cctx.tm().finisher().execute(this, () -> { + IgniteTxCommitFuture commitFut = startCommit(); - commitFuture.listen(f -> cctx.tm().finisher().execute(this, () -> { - try { - finishCommit(commitFuture); - } - catch (IgniteCheckedException e) { - U.error(log, "Failed to commit remote transaction: " + this, e); + commitFut.listen(f -> cctx.tm().finisher().execute(this, () -> { + try { + finishCommit(commitFut); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to commit remote transaction: " + this, e); - invalidate(true); - systemInvalidate(true); + invalidate(true); + systemInvalidate(true); - rollbackRemoteTx(); - } - })); + rollbackRemoteTx(); + } + })); + }); return true; } @@ -714,7 +716,7 @@ private Collection commitEntries() { return near() || cctx.snapshot().needTxReadLogging() ? allEntries() : writeEntries(); } - public void finishCommit(IgniteTxCommitEntriesFuture commitEntriesFuture) throws IgniteCheckedException { + private void finishCommitEntries(IgniteTxCommitEntriesFuture commitEntriesFuture) throws IgniteCheckedException { assert commitEntriesFuture.isDone(); // Nothing to commit. @@ -778,7 +780,7 @@ public void finishCommit(IgniteTxCommitEntriesFuture commitEntriesFuture) throws throw (Error)ex; } - if (err != null) { + if (err == null) { if (latestPtr != null && !cctx.tm().logTxRecords()) cctx.wal().flush(latestPtr, false); } @@ -802,6 +804,10 @@ public void finishCommit(IgniteTxCommitEntriesFuture commitEntriesFuture) throws /** {@inheritDoc} */ @Override public IgniteTxCommitFuture startCommit() { try { + if (!Thread.currentThread().getName().contains("dedicated")) { + throw new AssertionError("Commit requested not from dedicated stripe."); + } + if (optimistic()) state(PREPARED); @@ -838,7 +844,7 @@ public void finishCommit(IgniteTxCommitEntriesFuture commitEntriesFuture) throws commitFuture.get(); // Check for errors. if (commitFuture.commitEntriesFuture() != null) - finishCommit(commitFuture.commitEntriesFuture()); + finishCommitEntries(commitFuture.commitEntriesFuture()); } /** @@ -854,18 +860,20 @@ public void forceCommit() throws IgniteCheckedException { @Override public IgniteInternalFuture commitAsync() { GridFutureAdapter fut = new GridFutureAdapter<>(); - IgniteTxCommitFuture commitFut = startCommit(); + cctx.tm().finisher().execute(this, () -> { + IgniteTxCommitFuture commitFut = startCommit(); - commitFut.listen(f -> cctx.tm().finisher().execute(this, () -> { - try { - finishCommit(commitFut); + commitFut.listen(f -> cctx.tm().finisher().execute(this, () -> { + try { + finishCommit(commitFut); - fut.onDone(this); - } - catch (IgniteCheckedException e) { - fut.onDone(e); - } - })); + fut.onDone(this); + } + catch (IgniteCheckedException e) { + fut.onDone(e); + } + })); + }); return fut; } @@ -896,7 +904,7 @@ public void forceCommit() throws IgniteCheckedException { U.error(log, "Failed to invalidate transaction: " + xidVersion(), e); } - return null; + return new GridFinishedFuture<>(); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java index 837fca67d2015..82410e903183c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java @@ -40,7 +40,6 @@ import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxMapping; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareResponse; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; -import org.apache.ignite.internal.processors.cache.transactions.IgniteTxCommitEntriesFuture; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxCommitFuture; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter; @@ -788,23 +787,11 @@ private IgniteInternalFuture obtainLockAsync( } } -/* - log.warning("Finish thread id = " + Thread.currentThread().getName()); -*/ - - if (!Thread.currentThread().getName().contains("dedicated")) { + if (!Thread.currentThread().getName().contains("dedicated")) throw new AssertionError("Commit requested not from dedicated stipe"); - } - - if (commit && !isRollbackOnly()) { - IgniteTxCommitEntriesFuture fut = startCommit(); - IgniteTxCommitFuture commitFuture = new IgniteTxCommitFuture(fut, true); - - fut.listen(f -> commitFuture.onDone()); - - return commitFuture; - } + if (commit && !isRollbackOnly()) + return new IgniteTxCommitFuture(startCommit(), true); else return new IgniteTxCommitFuture(userRollback(clearThreadMap), false); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java index 1d4bf3ed6aff4..3bbf82cbbd970 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java @@ -62,7 +62,6 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtDetachedCacheEntry; import org.apache.ignite.internal.processors.cache.dr.GridCacheDrInfo; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; -import org.apache.ignite.internal.processors.cache.transactions.IgniteTxCommitEntriesFuture; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxCommitFuture; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; @@ -3186,19 +3185,11 @@ private void readyNearLock(IgniteTxEntry txEntry, log.warning("Finish thread id = " + Thread.currentThread().getName()); */ - if (!Thread.currentThread().getName().contains("dedicated")) { + if (!Thread.currentThread().getName().contains("dedicated")) throw new AssertionError("Commit requested not from dedicated stipe"); - } - - if (commit && !isRollbackOnly()) { - IgniteTxCommitEntriesFuture fut = startCommit(); - - IgniteTxCommitFuture commitFuture = new IgniteTxCommitFuture(fut, true); - fut.listen(f -> commitFuture.onDone()); - - return commitFuture; - } + if (commit && !isRollbackOnly()) + return new IgniteTxCommitFuture(startCommit(), true); else return new IgniteTxCommitFuture(userRollback(clearThreadMap), false); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxCommitFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxCommitFuture.java index 6115af00662fa..07c94e9674430 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxCommitFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxCommitFuture.java @@ -16,6 +16,8 @@ public IgniteTxCommitFuture(IgniteTxCommitEntriesFuture commitEntriesFuture, boo if (!async) onDone(); + else + commitEntriesFuture.listen(f -> onDone()); } public IgniteTxCommitFuture(Throwable err) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java index a509a7c55aa7c..60b4a1ed76861 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java @@ -1098,20 +1098,16 @@ else if (nearTx != null) res.invalidPartitionsByCacheId(dhtTx.invalidPartitions()); if (req.onePhaseCommit()) { - assert req.last(); + GridDhtTxRemote remoteTx = dhtTx; + GridNearTxRemote nearRemoteTx = nearTx; + GridDhtTxPrepareResponse prepRes = res; - if (dhtTx != null) { - dhtTx.onePhaseCommit(true); - dhtTx.needReturnValue(req.needReturnValue()); + if (dhtTx != null) + ctx.tm().finisher().execute(dhtTx, () -> processPrepareOnPhase(nodeId, req, remoteTx, nearRemoteTx, prepRes)); + else if (nearTx != null) + ctx.tm().finisher().execute(nearTx, () -> processPrepareOnPhase(nodeId, req, remoteTx, nearRemoteTx, prepRes)); - finish(dhtTx, req); - } - - if (nearTx != null) { - nearTx.onePhaseCommit(true); - - finish(nearTx, req); - } + return; } } catch (IgniteCheckedException e) { @@ -1145,7 +1141,84 @@ else if (e instanceof IgniteTxHeuristicCheckedException) { req.deployInfo() != null); } - if (req.onePhaseCommit()) { + sendReply(nodeId, req, res, dhtTx, nearTx); + + assert req.txState() != null || res.error() != null || (dhtTx == null && nearTx == null) : + req + " tx=" + dhtTx + " nearTx=" + nearTx; + } + + private void processPrepareOnPhase(final UUID nodeId, final GridDhtTxPrepareRequest req, GridDhtTxRemote dhtTx, GridNearTxRemote nearTx, final GridDhtTxPrepareResponse res) { + assert req.onePhaseCommit(); + assert req.last(); + + GridCompoundFuture unifiedFut = new GridCompoundFuture(); + + IgniteTxCommitFuture dhtTxCommitFut = null; + IgniteTxCommitFuture nearTxCommitFut = null; + + if (dhtTx != null) { + dhtTx.onePhaseCommit(true); + dhtTx.needReturnValue(req.needReturnValue()); + + dhtTxCommitFut = startOnePhaseCommit(dhtTx, req); + + unifiedFut.add(dhtTxCommitFut); + } + + if (nearTx != null) { + nearTx.onePhaseCommit(true); + + nearTxCommitFut = startOnePhaseCommit(nearTx, req); + + unifiedFut.add(nearTxCommitFut); + } + + final IgniteTxCommitFuture dhtTxCommitFut0 = dhtTxCommitFut; + final IgniteTxCommitFuture nearTxCommitFut0 = nearTxCommitFut; + + unifiedFut.markInitialized(); + + unifiedFut.listen(f -> ctx.tm().finisher().execute(dhtTx, () -> { + GridDhtTxPrepareResponse response = res; + + try { + if (dhtTxCommitFut0 != null) + dhtTx.finishCommit(dhtTxCommitFut0); + + if (nearTxCommitFut0 != null) + nearTx.finishCommit(nearTxCommitFut0); + } + catch (IgniteCheckedException e) { + if (e instanceof IgniteTxRollbackCheckedException) + U.error(log, "Transaction was rolled back before prepare completed: " + req, e); + else if (e instanceof IgniteTxOptimisticCheckedException) { + if (log.isDebugEnabled()) + log.debug("Optimistic failure for remote transaction (will rollback): " + req); + } + else if (e instanceof IgniteTxHeuristicCheckedException) { + U.warn(log, "Failed to commit transaction (all transaction entries were invalidated): " + + CU.txString(dhtTx)); + } + else + U.error(log, "Failed to process prepare request: " + req, e); + + if (nearTx != null) + try { + nearTx.rollbackRemoteTx(); + } + catch (Throwable e1) { + e.addSuppressed(e1); + } + + response = new GridDhtTxPrepareResponse( + req.partition(), + req.version(), + req.futureId(), + req.miniId(), + e, + req.deployInfo() != null); + } + IgniteInternalFuture completeFut; IgniteInternalFuture dhtFin = dhtTx == null ? @@ -1168,7 +1241,7 @@ else if (e instanceof IgniteTxHeuristicCheckedException) { completeFut = dhtFin != null ? dhtFin : nearFin; if (completeFut != null) { - final GridDhtTxPrepareResponse res0 = res; + final GridDhtTxPrepareResponse res0 = response; final GridDhtTxRemote dhtTx0 = dhtTx; final GridNearTxRemote nearTx0 = nearTx; @@ -1179,13 +1252,8 @@ else if (e instanceof IgniteTxHeuristicCheckedException) { }); } else - sendReply(nodeId, req, res, dhtTx, nearTx); - } - else - sendReply(nodeId, req, res, dhtTx, nearTx); - - assert req.txState() != null || res.error() != null || (dhtTx == null && nearTx == null) : - req + " tx=" + dhtTx + " nearTx=" + nearTx; + sendReply(nodeId, req, response, dhtTx, nearTx); + })); } /** @@ -1247,47 +1315,51 @@ private void processDhtTxFinishRequest(final UUID nodeId, final GridDhtTxFinishR if (nearTx != null && nearTx.local()) nearTx = null; - finish(nodeId, dhtTx, req); + final GridNearTxRemote nearTx0 = nearTx; - if (nearTx != null) - finish(nodeId, nearTx, req); + ctx.tm().finisher().execute(dhtTx, () -> { + finish(nodeId, dhtTx, req); - if (req.replyRequired()) { - IgniteInternalFuture completeFut; + if (nearTx0 != null) + finish(nodeId, nearTx0, req); - IgniteInternalFuture dhtFin = dhtTx == null ? - null : dhtTx.done() ? null : dhtTx.finishFuture(); + if (req.replyRequired()) { + IgniteInternalFuture completeFut; - final IgniteInternalFuture nearFin = nearTx == null ? - null : nearTx.done() ? null : nearTx.finishFuture(); + IgniteInternalFuture dhtFin = dhtTx == null ? + null : dhtTx.done() ? null : dhtTx.finishFuture(); - if (dhtFin != null && nearFin != null) { - GridCompoundFuture fut = new GridCompoundFuture(); + final IgniteInternalFuture nearFin = nearTx0 == null ? + null : nearTx0.done() ? null : nearTx0.finishFuture(); - fut.add(dhtFin); - fut.add(nearFin); + if (dhtFin != null && nearFin != null) { + GridCompoundFuture fut = new GridCompoundFuture(); - fut.markInitialized(); + fut.add(dhtFin); + fut.add(nearFin); - completeFut = fut; - } - else - completeFut = dhtFin != null ? dhtFin : nearFin; + fut.markInitialized(); - if (completeFut != null) { - completeFut.listen(new CI1>() { - @Override public void apply(IgniteInternalFuture fut) { - sendReply(nodeId, req, true, nearTxId); - } - }); + completeFut = fut; + } + else + completeFut = dhtFin != null ? dhtFin : nearFin; + + if (completeFut != null) { + completeFut.listen(new CI1>() { + @Override public void apply(IgniteInternalFuture fut) { + sendReply(nodeId, req, true, nearTxId); + } + }); + } + else + sendReply(nodeId, req, true, nearTxId); } else - sendReply(nodeId, req, true, nearTxId); - } - else - sendReply(nodeId, req, true, null); + sendReply(nodeId, req, true, null); - assert req.txState() != null || (dhtTx == null && nearTx == null) : req + " tx=" + dhtTx + " nearTx=" + nearTx; + assert req.txState() != null || (dhtTx == null && nearTx0 == null) : req + " tx=" + dhtTx + " nearTx=" + nearTx0; + }); } /** @@ -1377,11 +1449,11 @@ else if (log.isDebugEnabled()) tx.invalidate(true); tx.systemInvalidate(true); - IgniteTxCommitFuture commitFuture = tx.startCommit(); + IgniteTxCommitFuture commitFut = tx.startCommit(); - commitFuture.listen(f -> ctx.tm().finisher().execute(tx, () -> { + commitFut.listen(f -> ctx.tm().finisher().execute(tx, () -> { try { - tx.finishCommit(commitFuture); + tx.finishCommit(commitFut); } catch (IgniteCheckedException ex) { U.error(log, "Failed to invalidate transaction: " + tx, ex); @@ -1397,19 +1469,27 @@ else if (log.isDebugEnabled()) * @param tx Transaction. * @param req Request. */ - private void finish( + private IgniteTxCommitFuture startOnePhaseCommit( GridDistributedTxRemoteAdapter tx, - GridDhtTxPrepareRequest req) throws IgniteTxHeuristicCheckedException { + GridDhtTxPrepareRequest req + ) { assert tx != null : "No transaction for one-phase commit prepare request: " + req; - try { - tx.commitVersion(req.writeVersion()); - tx.invalidate(req.isInvalidate()); + tx.commitVersion(req.writeVersion()); + tx.invalidate(req.isInvalidate()); + + // Complete remote candidates. + tx.doneRemote(req.version(), null, null, null); - // Complete remote candidates. - tx.doneRemote(req.version(), null, null, null); + return tx.startCommit(); + } - tx.startCommit(); + private void finishOnePhaseCommit( + GridDistributedTxRemoteAdapter tx, + IgniteTxCommitFuture commitFuture + ) throws IgniteCheckedException { + try { + tx.finishCommit(commitFuture); } catch (IgniteTxHeuristicCheckedException e) { // Just rethrow this exception. Transaction was already uncommitted. @@ -1427,6 +1507,7 @@ private void finish( } catch (Throwable e1) { e.addSuppressed(e1); + U.error(log, "Failed to automatically rollback transaction: " + tx, e1); } From 69a4667ac7abc85eeb1ee98c5453b1f7ba2c6681 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 22 Aug 2018 20:06:32 +0300 Subject: [PATCH 12/18] IGNITE-9270 WIP. --- .../apache/ignite/IgniteSystemProperties.java | 2 ++ .../processors/cache/CacheEntryExecutor.java | 17 +++++++------ .../GridDistributedTxRemoteAdapter.java | 7 +++--- .../dht/GridDhtTxLocalAdapter.java | 3 +-- .../distributed/near/GridNearTxLocal.java | 3 +-- .../cache/transactions/IgniteTxFinisher.java | 25 +++++++++++++++++++ 6 files changed, 42 insertions(+), 15 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index 02c8b30637f54..7fd6a6b101eed 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -949,6 +949,8 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_EVICTION_PERMITS = "IGNITE_EVICTION_PERMITS"; + public static final String IGNITE_ENABLE_THREAD_PER_PARTITION = "IGNITE_ENABLE_THREAD_PER_PARTITION"; + /** * Enforces singleton. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryExecutor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryExecutor.java index a460ed891589d..50ac3a086f5a1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryExecutor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryExecutor.java @@ -1,10 +1,12 @@ package org.apache.ignite.internal.processors.cache; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.internal.util.StripedExecutor; import org.jetbrains.annotations.Nullable; public class CacheEntryExecutor { + private static final boolean TPP_ENABLED = IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_ENABLE_THREAD_PER_PARTITION, true); private final StripedExecutor executor; @@ -31,7 +33,7 @@ public CacheEntryOperationFuture execute( CacheEntryOperationFuture future = new CacheEntryOperationFuture<>(); - executor.execute(entry.key().partition(), () -> { + Runnable cmd = () -> { GridCacheEntryEx entry0 = entry; R result; @@ -77,14 +79,13 @@ public CacheEntryOperationFuture execute( cctx.database().checkpointReadUnlock(); } } - }); + }; - return future; - } - - static class ExecutionContext { - - private GridCacheEntryEx entry; + if (TPP_ENABLED) + executor.execute(entry.key().partition(), cmd); + else + cmd.run(); + return future; } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java index bb99f76c9acbc..62a3d699d019c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java @@ -719,6 +719,9 @@ private Collection commitEntries() { private void finishCommitEntries(IgniteTxCommitEntriesFuture commitEntriesFuture) throws IgniteCheckedException { assert commitEntriesFuture.isDone(); + if (state() == COMMITTED) + return; + // Nothing to commit. if (!commitEntriesFuture.initialized()) return; @@ -804,9 +807,7 @@ private void finishCommitEntries(IgniteTxCommitEntriesFuture commitEntriesFuture /** {@inheritDoc} */ @Override public IgniteTxCommitFuture startCommit() { try { - if (!Thread.currentThread().getName().contains("dedicated")) { - throw new AssertionError("Commit requested not from dedicated stripe."); - } + cctx.tm().finisher().check(); if (optimistic()) state(PREPARED); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java index 82410e903183c..f5c0fbf6e1dc5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java @@ -787,8 +787,7 @@ private IgniteInternalFuture obtainLockAsync( } } - if (!Thread.currentThread().getName().contains("dedicated")) - throw new AssertionError("Commit requested not from dedicated stipe"); + cctx.tm().finisher().check(); if (commit && !isRollbackOnly()) return new IgniteTxCommitFuture(startCommit(), true); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java index 3bbf82cbbd970..cb4a45d9088f9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java @@ -3185,8 +3185,7 @@ private void readyNearLock(IgniteTxEntry txEntry, log.warning("Finish thread id = " + Thread.currentThread().getName()); */ - if (!Thread.currentThread().getName().contains("dedicated")) - throw new AssertionError("Commit requested not from dedicated stipe"); + cctx.tm().finisher().check(); if (commit && !isRollbackOnly()) return new IgniteTxCommitFuture(startCommit(), true); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java index 0d3a0d52c0516..d0d8ba7a7eba1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java @@ -5,6 +5,7 @@ import java.util.List; import java.util.Map; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.jsr166.ConcurrentLinkedHashMap; @@ -13,6 +14,8 @@ * */ public class IgniteTxFinisher { + private static final boolean TPP_ENABLED = IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_ENABLE_THREAD_PER_PARTITION, true); + private static final int DEDICATED_WORKER_IDX = 0; private final IgniteLogger log; @@ -35,6 +38,12 @@ public IgniteTxFinisher(GridCacheSharedContext cctx) { } public void execute(IgniteInternalTx tx, Runnable transactionOp) { + if (!TPP_ENABLED) { + transactionOp.run(); + + return; + } + cctx.kernalContext().getStripedExecutorService().executeDedicated(DEDICATED_WORKER_IDX, () -> { GridCacheVersion txId = tx.xidVersion(); @@ -60,6 +69,12 @@ public void execute(IgniteInternalTx tx, Runnable transactionOp) { } public void send(IgniteInternalTx tx, Runnable transactionSendOp) { + if (!TPP_ENABLED) { + transactionSendOp.run(); + + return; + } + GridCacheVersion txId = tx.xidVersion(); long order = txOrdering.get(txId); @@ -82,6 +97,9 @@ public void send(IgniteInternalTx tx, Runnable transactionSendOp) { } public void finishSend(IgniteInternalTx tx) { + if (!TPP_ENABLED) + return; + GridCacheVersion txId = tx.xidVersion(); long order = txOrdering.get(txId); @@ -119,4 +137,11 @@ public void finishSend(IgniteInternalTx tx) { public long order(IgniteInternalTx tx) { return txOrdering.get(tx.xidVersion()); } + + public void check() { + if (TPP_ENABLED) { + if (!Thread.currentThread().getName().contains("dedicated")) + throw new AssertionError("Commit requested not from dedicated stripe."); + } + } } From 182ceb1bf9b060a83588dbd7b307708804e5980e Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 22 Aug 2018 20:08:45 +0300 Subject: [PATCH 13/18] IGNITE-9270 WIP. --- .../cache/distributed/GridDistributedTxRemoteAdapter.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java index 62a3d699d019c..20fbb2bc4f250 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java @@ -468,7 +468,7 @@ private IgniteTxCommitEntriesFuture startCommitEntries() { // Node that for near transactions we grab all entries. for (IgniteTxEntry txEntry : entries) { // Prepare context for transaction entry. - TransactionEntryContext txContext = prepareContext(txEntry); + TransactionEntryContext txContext = prepareContext(txEntry, ret); // Nothing to perform. if (txContext == null) { @@ -989,7 +989,7 @@ private GridCacheEntryEx refresh(IgniteTxEntry txEntry) { return txEntry.context().cache().entryEx(txEntry.key(), topologyVersion()); } - private TransactionEntryContext prepareContext(IgniteTxEntry txEntry) throws IgniteCheckedException { + private TransactionEntryContext prepareContext(IgniteTxEntry txEntry, GridCacheReturn ret) throws IgniteCheckedException { GridCacheContext cacheCtx = txEntry.context(); for (;;) { @@ -1012,7 +1012,7 @@ private TransactionEntryContext prepareContext(IgniteTxEntry txEntry) throws Ign if (!F.isEmpty(txEntry.entryProcessors())) txEntry.cached().unswap(false); - IgniteBiTuple res = applyTransformClosures(txEntry, false, null); + IgniteBiTuple res = applyTransformClosures(txEntry, false, ret); GridCacheOperation op = res.get1(); CacheObject val = res.get2(); From 1f3f455adc58fbd02baa8f2547530bc3a24dd42d Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Fri, 24 Aug 2018 14:46:25 +0300 Subject: [PATCH 14/18] IGNITE-9270 WIP. --- .../apache/ignite/internal/IgnitionEx.java | 2 +- .../processors/cache/CacheEntryExecutor.java | 6 +- .../GridDistributedTxRemoteAdapter.java | 94 +++++++++++++----- .../dht/GridDhtTxPrepareFuture.java | 2 + .../near/GridNearTxFinishFuture.java | 11 ++- .../distributed/near/GridNearTxLocal.java | 41 ++++---- .../cache/transactions/IgniteTxAdapter.java | 9 +- .../IgniteTxCommitEntriesFuture.java | 12 +++ .../cache/transactions/IgniteTxFinisher.java | 10 +- .../cache/transactions/IgniteTxHandler.java | 96 ++++++++++++------- 10 files changed, 192 insertions(+), 91 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java index 75f6896c6d26a..929e31e37bdcc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java @@ -1831,7 +1831,7 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { } }, workerRegistry, - 1); + 2); // Note that since we use 'LinkedBlockingQueue', number of // maximum threads has no effect. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryExecutor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryExecutor.java index 50ac3a086f5a1..2e984e8249564 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryExecutor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryExecutor.java @@ -1,6 +1,5 @@ package org.apache.ignite.internal.processors.cache; -import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.internal.util.StripedExecutor; import org.jetbrains.annotations.Nullable; @@ -12,7 +11,6 @@ public class CacheEntryExecutor { private final GridCacheSharedContext cctx; - public CacheEntryExecutor(StripedExecutor executor, GridCacheSharedContext cctx) { this.executor = executor; this.cctx = cctx; @@ -58,7 +56,7 @@ public CacheEntryOperationFuture execute( assert entry0 != null; } - catch (IgniteCheckedException e) { + catch (Throwable e) { future.onDone(e); break; @@ -70,7 +68,7 @@ public CacheEntryOperationFuture execute( break; } } - catch (IgniteCheckedException e) { + catch (Throwable e) { future.onDone(e); break; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java index 20fbb2bc4f250..fab0132b7d266 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java @@ -331,23 +331,21 @@ private void doneRemote(IgniteTxEntry txEntry, if (!hasWriteKey(entry.txKey())) return false; - cctx.tm().finisher().execute(this, () -> { - IgniteTxCommitFuture commitFut = startCommit(); + IgniteTxCommitEntriesFuture commitFut = startCommitEntries(); - commitFut.listen(f -> cctx.tm().finisher().execute(this, () -> { - try { - finishCommit(commitFut); - } - catch (IgniteCheckedException e) { - U.error(log, "Failed to commit remote transaction: " + this, e); + commitFut.listen(f -> cctx.tm().finisher().execute(this, () -> { + try { + finishCommitEntries(commitFut); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to commit remote transaction: " + this, e); - invalidate(true); - systemInvalidate(true); + invalidate(true); + systemInvalidate(true); - rollbackRemoteTx(); - } - })); - }); + rollbackRemoteTx(); + } + })); return true; } @@ -420,18 +418,31 @@ public final void prepareRemoteTx() throws IgniteCheckedException { */ @SuppressWarnings({"CatchGenericClass"}) private IgniteTxCommitEntriesFuture startCommitEntries() { - if (state() != COMMITTING) + if (state() != COMMITTING) { +// U.dumpStack(log, "Start commit has end due to not COMMITING state -> " + this); + return IgniteTxCommitEntriesFuture.FINISHED; + } + + if (checkLocks()) { + log.warning("Start commit has end due to locks -> " + this); - if (checkLocks()) return IgniteTxCommitEntriesFuture.FINISHED; + } Map writeMap = txState.writeMap(); - if (F.isEmpty(writeMap)) + if (!COMMIT_ALLOWED_UPD.compareAndSet(this, 0, 1)) { + log.warning("Start commit has end due to twice entered -> " + this); + return IgniteTxCommitEntriesFuture.FINISHED; + } - IgniteCheckedException err = null; + if (F.isEmpty(writeMap)) { + log.warning("Write map is empty -> " + this); + + return IgniteTxCommitEntriesFuture.EMPTY; + } GridCacheReturnCompletableWrapper wrapper = null; @@ -662,6 +673,8 @@ private IgniteTxCommitEntriesFuture startCommitEntries() { } } catch (Throwable t) { + commitEntriesFut.markInitialized(); + commitEntriesFut.onDone(t); return commitEntriesFut; @@ -719,12 +732,20 @@ private Collection commitEntries() { private void finishCommitEntries(IgniteTxCommitEntriesFuture commitEntriesFuture) throws IgniteCheckedException { assert commitEntriesFuture.isDone(); - if (state() == COMMITTED) + cctx.tm().finisher().check(); + + if (state() == COMMITTED) { + log.warning("Already committed"); + return; + } // Nothing to commit. - if (!commitEntriesFuture.initialized()) + if (!commitEntriesFuture.initialized()) { + log.warning("Nothing to commit -> " + commitEntriesFuture); + return; + } IgniteCheckedException err = null; @@ -812,6 +833,8 @@ private void finishCommitEntries(IgniteTxCommitEntriesFuture commitEntriesFuture if (optimistic()) state(PREPARED); + log.warning("I'm here 1"); + if (!state(COMMITTING)) { TransactionState state = state(); @@ -832,6 +855,8 @@ private void finishCommitEntries(IgniteTxCommitEntriesFuture commitEntriesFuture return new IgniteTxCommitFuture(true); } + log.warning("I'm here 2"); + return new IgniteTxCommitFuture(startCommitEntries(), true); } catch (Throwable t) { @@ -839,13 +864,14 @@ private void finishCommitEntries(IgniteTxCommitEntriesFuture commitEntriesFuture } } - @Override public void finishCommit(IgniteTxCommitFuture commitFuture) throws IgniteCheckedException { - assert commitFuture.isDone(); + /** {@inheritDoc} */ + @Override public void finishCommit(IgniteTxCommitFuture commitFut) throws IgniteCheckedException { + assert commitFut.isDone(); - commitFuture.get(); // Check for errors. + commitFut.get(); // Check for errors. - if (commitFuture.commitEntriesFuture() != null) - finishCommitEntries(commitFuture.commitEntriesFuture()); + if (commitFut.commitEntriesFuture() != null) + finishCommitEntries(commitFut.commitEntriesFuture()); } /** @@ -854,7 +880,23 @@ private void finishCommitEntries(IgniteTxCommitEntriesFuture commitEntriesFuture * @throws IgniteCheckedException If commit failed. */ public void forceCommit() throws IgniteCheckedException { - commitAsync(); + log.warning("Commit forced"); + + IgniteTxCommitEntriesFuture commitFut = startCommitEntries(); + + commitFut.listen(f -> cctx.tm().finisher().execute(this, () -> { + try { + finishCommitEntries(commitFut); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to commit remote transaction: " + this, e); + + invalidate(true); + systemInvalidate(true); + + rollbackRemoteTx(); + } + })); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java index ba1210e24082c..bd7593fcccf29 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java @@ -1744,6 +1744,8 @@ void onNodeLeft() { */ @SuppressWarnings("ThrowableResultOfMethodCallIgnored") void onResult(GridDhtTxPrepareResponse res) { + log.warning("Responsed on -> " + this); + if (res.error() != null) // Fail the whole compound future. onError(res.error()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java index 3315e632882de..1a8f763cd6626 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java @@ -411,9 +411,7 @@ public void finish(final boolean commit, final boolean clearThreadMap, final boo if (tx.onNeedCheckBackup()) { assert tx.onePhaseCommit(); -/* log.warning("One-phase commit -> " + cctx.tm().finisher().order(tx) + " " + tx); -*/ checkBackup(); @@ -478,9 +476,7 @@ private void tryRollbackAsync(boolean onTimeout) { private void doFinish(boolean commit, boolean clearThreadMap) { IgniteTxCommitFuture commitFuture = tx.startLocalCommit(commit, clearThreadMap); -/* log.warning("Local commit started " + cctx.tm().finisher().order(tx) + " " + commitFuture + " " + commitFuture.started() + " " + commitFuture.async() + " " + commitFuture.commitEntriesFuture()); -*/ if (commitFuture.async()) commitFuture.listen(f -> cctx.tm().finisher().execute(tx, () -> doFinishAsync(commitFuture, commit, clearThreadMap))); @@ -496,6 +492,8 @@ private void doFinish(boolean commit, boolean clearThreadMap) { */ private void doFinishAsync(IgniteTxCommitFuture commitFuture, boolean commit, boolean clearThreadMap) { try { + log.warning("Local commit finished -> " + cctx.tm().finisher().order(tx) + " " + commitFuture + " " + commitFuture.started()); + if (commitFuture.started()) { tx.finishLocalCommit(commitFuture, commit, clearThreadMap); @@ -521,6 +519,8 @@ private void doFinishAsync(IgniteTxCommitFuture commitFuture, boolean commit, bo finish(mappings.mappings(), commit, !clearThreadMap); } + cctx.tm().finisher().finishSend(tx); + markInitialized(); } else @@ -536,7 +536,8 @@ private void doFinishAsync(IgniteTxCommitFuture commitFuture, boolean commit, bo onDone(e); } finally { - cctx.tm().finisher().finishSend(tx); + if (!initialized()) + cctx.tm().finisher().finishSend(tx); if (commit && tx.onePhaseCommit() && diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java index cb4a45d9088f9..53ae83cab1eaf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java @@ -3296,27 +3296,34 @@ public IgniteInternalFuture commitNearTxLocalAsync() { final IgniteInternalFuture prepareFut = prepareNearTxLocal(); - prepareFut.listen(f -> cctx.tm().finisher().execute(this, () -> { - try { - // Make sure that here are no exceptions. - prepareFut.get(); + log.warning("I'm here 7 -> " + prepareFut); - fut0.finish(true, true, false); - } - catch (Error | RuntimeException e) { - COMMIT_ERR_UPD.compareAndSet(GridNearTxLocal.this, null, e); + prepareFut.listen(f -> { + log.warning("I'm here 8"); - fut0.finish(false, true, false); + cctx.tm().finisher().execute(this, () -> { + try { + // Make sure that here are no exceptions. + prepareFut.get(); - throw e; - } - catch (IgniteCheckedException e) { - COMMIT_ERR_UPD.compareAndSet(GridNearTxLocal.this, null, e); + log.warning("I'm here 8"); - if (!(e instanceof NodeStoppingException)) - fut0.finish(false, true, true); - } - })); + fut0.finish(true, true, false); + } catch (Error | RuntimeException e) { + COMMIT_ERR_UPD.compareAndSet(GridNearTxLocal.this, null, e); + + fut0.finish(false, true, false); + + throw e; + } catch (IgniteCheckedException e) { + COMMIT_ERR_UPD.compareAndSet(GridNearTxLocal.this, null, e); + + if (!(e instanceof NodeStoppingException)) + fut0.finish(false, true, true); + } + }); + + }); return fut0; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java index 6fdb046117501..b24630f71b391 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java @@ -1170,7 +1170,7 @@ protected final boolean state(TransactionState state, boolean timedOut) { } /** */ - private void recordStateChangedEvent(TransactionState state){ + private void recordStateChangedEvent(TransactionState state) { if (!near() || !local()) // Covers only GridNearTxLocal's state changes. return; @@ -1534,7 +1534,8 @@ else if (log.isDebugEnabled()) protected IgniteBiTuple applyTransformClosures( IgniteTxEntry txEntry, boolean metrics, - @Nullable GridCacheReturn ret) throws GridCacheEntryRemovedException, IgniteCheckedException { + @Nullable GridCacheReturn ret + ) throws GridCacheEntryRemovedException, IgniteCheckedException { assert txEntry.op() != TRANSFORM || !F.isEmpty(txEntry.entryProcessors()) : txEntry; GridCacheContext cacheCtx = txEntry.context(); @@ -1682,8 +1683,8 @@ protected IgniteBiTuple con IgniteTxEntry txEntry, CacheObject newVal, GridCacheVersion newVer, - GridCacheEntryEx old) - throws IgniteCheckedException, GridCacheEntryRemovedException { + GridCacheEntryEx old + ) throws IgniteCheckedException, GridCacheEntryRemovedException { assert newVer != null; // 1. Calculate TTL and expire time. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxCommitEntriesFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxCommitEntriesFuture.java index bd868875ee846..7e1b1a2ce9196 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxCommitEntriesFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxCommitEntriesFuture.java @@ -7,6 +7,18 @@ public class IgniteTxCommitEntriesFuture extends GridCompoundFuture { + boolean remoteTx = (tx instanceof GridDhtTxRemote) || (tx instanceof GridNearTxRemote); + + cctx.kernalContext().getStripedExecutorService().executeDedicated(remoteTx ? REMOTE_TX_DEDICATED_WORKER_IDX : DEDICATED_WORKER_IDX, () -> { GridCacheVersion txId = tx.xidVersion(); - if (txId != null) { + if (!remoteTx) { boolean isNew = !txOrdering.containsKey(txId); if (!txOrdering.containsKey(txId)) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java index 60b4a1ed76861..da021bab4b8b2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java @@ -69,6 +69,7 @@ import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException; import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridFinishedFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.typedef.C1; import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.CI2; @@ -1102,12 +1103,16 @@ else if (nearTx != null) GridNearTxRemote nearRemoteTx = nearTx; GridDhtTxPrepareResponse prepRes = res; - if (dhtTx != null) + if (dhtTx != null) { ctx.tm().finisher().execute(dhtTx, () -> processPrepareOnPhase(nodeId, req, remoteTx, nearRemoteTx, prepRes)); - else if (nearTx != null) + + return; + } + else if (nearTx != null) { ctx.tm().finisher().execute(nearTx, () -> processPrepareOnPhase(nodeId, req, remoteTx, nearRemoteTx, prepRes)); - return; + return; + } } } catch (IgniteCheckedException e) { @@ -1251,8 +1256,9 @@ else if (e instanceof IgniteTxHeuristicCheckedException) { } }); } - else + else { sendReply(nodeId, req, response, dhtTx, nearTx); + } })); } @@ -1318,45 +1324,53 @@ private void processDhtTxFinishRequest(final UUID nodeId, final GridDhtTxFinishR final GridNearTxRemote nearTx0 = nearTx; ctx.tm().finisher().execute(dhtTx, () -> { - finish(nodeId, dhtTx, req); + GridCompoundFuture colocatedFut = new GridCompoundFuture(); + + colocatedFut.add(finish(nodeId, dhtTx, req)); if (nearTx0 != null) - finish(nodeId, nearTx0, req); + colocatedFut.add(finish(nodeId, nearTx0, req)); + + colocatedFut.markInitialized(); if (req.replyRequired()) { - IgniteInternalFuture completeFut; + colocatedFut.listen(f -> { + IgniteInternalFuture completeFut; - IgniteInternalFuture dhtFin = dhtTx == null ? - null : dhtTx.done() ? null : dhtTx.finishFuture(); + IgniteInternalFuture dhtFin = dhtTx == null ? + null : dhtTx.done() ? null : dhtTx.finishFuture(); - final IgniteInternalFuture nearFin = nearTx0 == null ? - null : nearTx0.done() ? null : nearTx0.finishFuture(); + final IgniteInternalFuture nearFin = nearTx0 == null ? + null : nearTx0.done() ? null : nearTx0.finishFuture(); - if (dhtFin != null && nearFin != null) { - GridCompoundFuture fut = new GridCompoundFuture(); + if (dhtFin != null && nearFin != null) { + GridCompoundFuture fut = new GridCompoundFuture(); - fut.add(dhtFin); - fut.add(nearFin); + fut.add(dhtFin); + fut.add(nearFin); - fut.markInitialized(); + fut.markInitialized(); - completeFut = fut; - } - else - completeFut = dhtFin != null ? dhtFin : nearFin; + completeFut = fut; + } + else + completeFut = dhtFin != null ? dhtFin : nearFin; - if (completeFut != null) { - completeFut.listen(new CI1>() { - @Override public void apply(IgniteInternalFuture fut) { - sendReply(nodeId, req, true, nearTxId); - } - }); - } - else - sendReply(nodeId, req, true, nearTxId); + if (completeFut != null) { + completeFut.listen(new CI1>() { + @Override public void apply(IgniteInternalFuture fut) { + sendReply(nodeId, req, true, nearTxId); + } + }); + } + else { + sendReply(nodeId, req, true, nearTxId); + } + }); } - else + else { sendReply(nodeId, req, true, null); + } assert req.txState() != null || (dhtTx == null && nearTx0 == null) : req + " tx=" + dhtTx + " nearTx=" + nearTx0; }); @@ -1367,7 +1381,7 @@ private void processDhtTxFinishRequest(final UUID nodeId, final GridDhtTxFinishR * @param tx Transaction. * @param req Request. */ - protected void finish( + protected IgniteInternalFuture finish( UUID nodeId, IgniteTxRemoteEx tx, GridDhtTxFinishRequest req @@ -1387,7 +1401,7 @@ protected void finish( log.debug("Received finish request for non-existing transaction (added to completed set) " + "[senderNodeId=" + nodeId + ", res=" + req + ']'); - return; + return new GridFinishedFuture<>(); } else if (log.isDebugEnabled()) log.debug("Received finish request for transaction [senderNodeId=" + nodeId + ", req=" + req + @@ -1409,9 +1423,13 @@ else if (log.isDebugEnabled()) IgniteTxCommitFuture commitFut = tx.startCommit(); + GridFutureAdapter fut = new GridFutureAdapter(); + commitFut.listen(f -> ctx.tm().finisher().execute(tx, () -> { try { tx.finishCommit(commitFut); + + fut.onDone(); } catch (Throwable e) { U.error(log, "Failed completing transaction [commit=" + req.commit() + ", tx=" + tx + ']', e); @@ -1429,17 +1447,24 @@ else if (log.isDebugEnabled()) catch (IgniteCheckedException ex) { U.error(log, "Failed to invalidate transaction: " + tx, ex); } + finally { + fut.onDone(); + } })); if (e instanceof Error) throw (Error)e; } })); + + return fut; } else { tx.doneRemote(req.baseVersion(), null, null, null); tx.rollbackRemoteTx(); + + return new GridFinishedFuture<>(); } } catch (Throwable e) { @@ -1449,6 +1474,8 @@ else if (log.isDebugEnabled()) tx.invalidate(true); tx.systemInvalidate(true); + GridFutureAdapter fut = new GridFutureAdapter(); + IgniteTxCommitFuture commitFut = tx.startCommit(); commitFut.listen(f -> ctx.tm().finisher().execute(tx, () -> { @@ -1458,10 +1485,15 @@ else if (log.isDebugEnabled()) catch (IgniteCheckedException ex) { U.error(log, "Failed to invalidate transaction: " + tx, ex); } + finally { + fut.onDone(); + } })); if (e instanceof Error) throw (Error)e; + + return fut; } } From 5e635f9b617016233b70cc88acebbf11e725f0bf Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Fri, 24 Aug 2018 15:20:06 +0300 Subject: [PATCH 15/18] IGNITE-9270 WIP. --- .../GridDistributedTxRemoteAdapter.java | 16 ++++++++-------- .../distributed/dht/GridDhtTxPrepareFuture.java | 2 -- .../distributed/near/GridNearTxFinishFuture.java | 6 +++--- .../cache/distributed/near/GridNearTxLocal.java | 6 ------ .../cache/transactions/IgniteTxFinisher.java | 2 ++ .../cache/transactions/IgniteTxHandler.java | 13 ++++++++++++- 6 files changed, 25 insertions(+), 20 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java index fab0132b7d266..55b7d157e0640 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java @@ -425,7 +425,7 @@ private IgniteTxCommitEntriesFuture startCommitEntries() { } if (checkLocks()) { - log.warning("Start commit has end due to locks -> " + this); +// log.warning("Start commit has end due to locks -> " + this); return IgniteTxCommitEntriesFuture.FINISHED; } @@ -433,13 +433,13 @@ private IgniteTxCommitEntriesFuture startCommitEntries() { Map writeMap = txState.writeMap(); if (!COMMIT_ALLOWED_UPD.compareAndSet(this, 0, 1)) { - log.warning("Start commit has end due to twice entered -> " + this); +// log.warning("Start commit has end due to twice entered -> " + this); return IgniteTxCommitEntriesFuture.FINISHED; } if (F.isEmpty(writeMap)) { - log.warning("Write map is empty -> " + this); +// log.warning("Write map is empty -> " + this); return IgniteTxCommitEntriesFuture.EMPTY; } @@ -735,14 +735,14 @@ private void finishCommitEntries(IgniteTxCommitEntriesFuture commitEntriesFuture cctx.tm().finisher().check(); if (state() == COMMITTED) { - log.warning("Already committed"); +// log.warning("Already committed"); return; } // Nothing to commit. if (!commitEntriesFuture.initialized()) { - log.warning("Nothing to commit -> " + commitEntriesFuture); +// log.warning("Nothing to commit -> " + commitEntriesFuture); return; } @@ -833,7 +833,7 @@ private void finishCommitEntries(IgniteTxCommitEntriesFuture commitEntriesFuture if (optimistic()) state(PREPARED); - log.warning("I'm here 1"); +// log.warning("I'm here 1"); if (!state(COMMITTING)) { TransactionState state = state(); @@ -855,7 +855,7 @@ private void finishCommitEntries(IgniteTxCommitEntriesFuture commitEntriesFuture return new IgniteTxCommitFuture(true); } - log.warning("I'm here 2"); +// log.warning("I'm here 2"); return new IgniteTxCommitFuture(startCommitEntries(), true); } @@ -880,7 +880,7 @@ private void finishCommitEntries(IgniteTxCommitEntriesFuture commitEntriesFuture * @throws IgniteCheckedException If commit failed. */ public void forceCommit() throws IgniteCheckedException { - log.warning("Commit forced"); +// log.warning("Commit forced"); IgniteTxCommitEntriesFuture commitFut = startCommitEntries(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java index bd7593fcccf29..ba1210e24082c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java @@ -1744,8 +1744,6 @@ void onNodeLeft() { */ @SuppressWarnings("ThrowableResultOfMethodCallIgnored") void onResult(GridDhtTxPrepareResponse res) { - log.warning("Responsed on -> " + this); - if (res.error() != null) // Fail the whole compound future. onError(res.error()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java index 1a8f763cd6626..d84e336af0b2f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java @@ -411,7 +411,7 @@ public void finish(final boolean commit, final boolean clearThreadMap, final boo if (tx.onNeedCheckBackup()) { assert tx.onePhaseCommit(); - log.warning("One-phase commit -> " + cctx.tm().finisher().order(tx) + " " + tx); +// log.warning("One-phase commit -> " + cctx.tm().finisher().order(tx) + " " + tx); checkBackup(); @@ -476,7 +476,7 @@ private void tryRollbackAsync(boolean onTimeout) { private void doFinish(boolean commit, boolean clearThreadMap) { IgniteTxCommitFuture commitFuture = tx.startLocalCommit(commit, clearThreadMap); - log.warning("Local commit started " + cctx.tm().finisher().order(tx) + " " + commitFuture + " " + commitFuture.started() + " " + commitFuture.async() + " " + commitFuture.commitEntriesFuture()); +// log.warning("Local commit started " + cctx.tm().finisher().order(tx) + " " + commitFuture + " " + commitFuture.started() + " " + commitFuture.async() + " " + commitFuture.commitEntriesFuture()); if (commitFuture.async()) commitFuture.listen(f -> cctx.tm().finisher().execute(tx, () -> doFinishAsync(commitFuture, commit, clearThreadMap))); @@ -492,7 +492,7 @@ private void doFinish(boolean commit, boolean clearThreadMap) { */ private void doFinishAsync(IgniteTxCommitFuture commitFuture, boolean commit, boolean clearThreadMap) { try { - log.warning("Local commit finished -> " + cctx.tm().finisher().order(tx) + " " + commitFuture + " " + commitFuture.started()); +// log.warning("Local commit finished -> " + cctx.tm().finisher().order(tx) + " " + commitFuture + " " + commitFuture.started()); if (commitFuture.started()) { tx.finishLocalCommit(commitFuture, commit, clearThreadMap); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java index 53ae83cab1eaf..0dc24be335a80 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java @@ -3296,18 +3296,12 @@ public IgniteInternalFuture commitNearTxLocalAsync() { final IgniteInternalFuture prepareFut = prepareNearTxLocal(); - log.warning("I'm here 7 -> " + prepareFut); - prepareFut.listen(f -> { - log.warning("I'm here 8"); - cctx.tm().finisher().execute(this, () -> { try { // Make sure that here are no exceptions. prepareFut.get(); - log.warning("I'm here 8"); - fut0.finish(true, true, false); } catch (Error | RuntimeException e) { COMMIT_ERR_UPD.compareAndSet(GridNearTxLocal.this, null, e); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java index 31076f949839a..3e70f4f048747 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxFinisher.java @@ -42,6 +42,8 @@ public IgniteTxFinisher(GridCacheSharedContext cctx) { } public void execute(IgniteInternalTx tx, Runnable transactionOp) { + assert tx != null; + if (!TPP_ENABLED) { transactionOp.run(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java index da021bab4b8b2..4852174f242de 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java @@ -1323,7 +1323,18 @@ private void processDhtTxFinishRequest(final UUID nodeId, final GridDhtTxFinishR final GridNearTxRemote nearTx0 = nearTx; - ctx.tm().finisher().execute(dhtTx, () -> { + if (dhtTx == null && nearTx == null) { + if (req.replyRequired()) { + sendReply(nodeId, req, true, nearTxId); + } + else { + sendReply(nodeId, req, true, null); + } + + return; + } + + ctx.tm().finisher().execute(dhtTx != null ? dhtTx : nearTx, () -> { GridCompoundFuture colocatedFut = new GridCompoundFuture(); colocatedFut.add(finish(nodeId, dhtTx, req)); From 05f5ef6003a8ea46704fab3749fa18e086fc0c38 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Fri, 24 Aug 2018 18:59:10 +0300 Subject: [PATCH 16/18] IGNITE-9270 Transaction context prepare moved inside per-partition thread. --- .../processors/cache/CacheEntryExecutor.java | 38 +- .../processors/cache/CacheEntryOperation.java | 2 +- .../GridDistributedTxRemoteAdapter.java | 453 ++++++------- .../transactions/IgniteTxLocalAdapter.java | 636 +++++++++--------- 4 files changed, 544 insertions(+), 585 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryExecutor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryExecutor.java index 2e984e8249564..e8dcf0aeba37a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryExecutor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryExecutor.java @@ -2,7 +2,6 @@ import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.internal.util.StripedExecutor; -import org.jetbrains.annotations.Nullable; public class CacheEntryExecutor { private static final boolean TPP_ENABLED = IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_ENABLE_THREAD_PER_PARTITION, true); @@ -21,53 +20,26 @@ public void stop(boolean cancel) { } public CacheEntryOperationFuture execute( - GridCacheEntryEx entry, - CacheEntryOperation operationClojure, - @Nullable CacheEntryRefresh refreshClojure, - @Nullable CacheEntryOperationCallback callbackClojure + int partitionId, + CacheEntryOperation operationClojure ) { - assert entry.key() != null : "Entry key is null [entry=" + entry + "]"; - assert entry.key().partition() != -1 : "Entry partition is undefined [entry=" + entry + "]"; + assert partitionId >= 0 : "Entry partition is undefined [partId=" + partitionId + "]"; CacheEntryOperationFuture future = new CacheEntryOperationFuture<>(); Runnable cmd = () -> { - GridCacheEntryEx entry0 = entry; - R result; for (;;) { cctx.database().checkpointReadLock(); try { - result = operationClojure.invoke(entry0); - - if (callbackClojure != null) - callbackClojure.invoke(entry0, result); + result = operationClojure.invoke(); future.onDone(result); break; } - catch (GridCacheEntryRemovedException re) { - if (refreshClojure != null) { - try { - entry0 = refreshClojure.refresh(entry0); - - assert entry0 != null; - } - catch (Throwable e) { - future.onDone(e); - - break; - } - } - else { - future.onDone(re); - - break; - } - } catch (Throwable e) { future.onDone(e); @@ -80,7 +52,7 @@ public CacheEntryOperationFuture execute( }; if (TPP_ENABLED) - executor.execute(entry.key().partition(), cmd); + executor.execute(partitionId, cmd); else cmd.run(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryOperation.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryOperation.java index 66f53e58f1163..8f27b166d6e4e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryOperation.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryOperation.java @@ -4,5 +4,5 @@ @FunctionalInterface public interface CacheEntryOperation { - public R invoke(GridCacheEntryEx entry) throws IgniteCheckedException, GridCacheEntryRemovedException; + public R invoke() throws IgniteCheckedException, GridCacheEntryRemovedException; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java index 55b7d157e0640..c19cee6fccf8b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java @@ -478,198 +478,190 @@ private IgniteTxCommitEntriesFuture startCommitEntries() { // Node that for near transactions we grab all entries. for (IgniteTxEntry txEntry : entries) { - // Prepare context for transaction entry. - TransactionEntryContext txContext = prepareContext(txEntry, ret); + GridCacheReturn ret0 = ret; - // Nothing to perform. - if (txContext == null) { - commitEntriesFut.add(new GridFinishedFuture<>()); - - continue; - } - - CacheEntryOperationFuture operationFut; - - if (txContext.operation == CREATE || txContext.operation == UPDATE) { - operationFut = cctx.cache().executor().execute(txContext.entry, e -> { - // Invalidate only for near nodes (backups cannot be invalidated). - if (isSystemInvalidate() || (isInvalidate() && txContext.cacheContext.isNear())) - return e.innerRemove(this, - eventNodeId(), - nodeId, - false, - true, - true, - txEntry.keepBinary(), - txEntry.hasOldValue(), - txEntry.oldValue(), - topVer, - null, - txContext.cacheContext.isDrEnabled() ? DR_BACKUP : DR_NONE, - near() ? null : txContext.explicitVer, - CU.subjectId(this, cctx), - resolveTaskName(), - txContext.dhtVer, - null); - else { - assert txContext.value != null : txEntry; - - return e.innerSet(this, - eventNodeId(), - nodeId, - txContext.value, - false, - false, - txEntry.ttl(), - true, - true, - txEntry.keepBinary(), - txEntry.hasOldValue(), - txEntry.oldValue(), - topVer, - null, - txContext.cacheContext.isDrEnabled() ? DR_BACKUP : DR_NONE, - txEntry.conflictExpireTime(), - near() ? null : txContext.explicitVer, - CU.subjectId(this, cctx), - resolveTaskName(), - txContext.dhtVer, - null); - } - }, - e -> refresh(txEntry), - (e, result) -> { - if (txContext.nearEntry == null) - return; - - // Keep near entry up to date. + CacheEntryOperationFuture operationFut = cctx.cache().executor().execute( + txEntry.key().partition(), + () -> { + for (;;) { try { - CacheObject valBytes = e.valueBytes(); - - txContext.nearEntry.updateOrEvict(xidVer, - valBytes, - e.expireTime(), - e.ttl(), - nodeId, - topVer); - } catch (GridCacheEntryRemovedException re) { - // This is impossible in TPP way, but anyway throw exception. - throw new IgniteCheckedException("Failed to get value bytes for entry: " + e, re); - } - }); - } else if (txContext.operation == DELETE) { - operationFut = cctx.cache().executor().execute(txContext.entry, e -> { - return e.innerRemove(this, - eventNodeId(), - nodeId, - false, - true, - true, - txEntry.keepBinary(), - txEntry.hasOldValue(), - txEntry.oldValue(), - topVer, - null, - txContext.cacheContext.isDrEnabled() ? DR_BACKUP : DR_NONE, - near() ? null : txContext.explicitVer, - CU.subjectId(this, cctx), - resolveTaskName(), - txContext.dhtVer, - txEntry.updateCounter() - ); - }, - e -> refresh(txEntry), - (e, result) -> { - // Keep near entry up to date. - if (txContext.nearEntry != null) - txContext.nearEntry.updateOrEvict(xidVer, null, 0, 0, nodeId, topVer); - }); - } else if (txContext.operation == RELOAD) { - operationFut = cctx.cache().executor().execute(txContext.entry, e -> { - CacheObject val = e.innerReload(); - - return new GridCacheUpdateTxResult(true, val, null); - }, - e -> refresh(txEntry), - (e, result) -> { - if (txContext.nearEntry == null) - return; - - // Keep near entry up to date. - GridNearCacheEntry nearEntry = txContext.nearEntry; - - for (; ; ) { - try { - CacheObject reloaded = nearEntry.innerReload(); - - nearEntry.updateOrEvict(e.version(), - reloaded, - e.expireTime(), - e.ttl(), + TransactionEntryContext txContext = prepareContext(txEntry, ret0); + + if (txContext == null) + return new GridCacheUpdateTxResult(false, null, null); + + GridCacheUpdateTxResult result; + + GridCacheEntryEx e = txContext.entry; + + if (txContext.operation == CREATE || txContext.operation == UPDATE) { + // Invalidate only for near nodes (backups cannot be invalidated). + if (isSystemInvalidate() || (isInvalidate() && txContext.cacheContext.isNear())) + result = e.innerRemove(this, + eventNodeId(), + nodeId, + false, + true, + true, + txEntry.keepBinary(), + txEntry.hasOldValue(), + txEntry.oldValue(), + topVer, + null, + txContext.cacheContext.isDrEnabled() ? DR_BACKUP : DR_NONE, + near() ? null : txContext.explicitVer, + CU.subjectId(this, cctx), + resolveTaskName(), + txContext.dhtVer, + null); + else { + assert txContext.value != null : txEntry; + + result = e.innerSet(this, + eventNodeId(), + nodeId, + txContext.value, + false, + false, + txEntry.ttl(), + true, + true, + txEntry.keepBinary(), + txEntry.hasOldValue(), + txEntry.oldValue(), + topVer, + null, + txContext.cacheContext.isDrEnabled() ? DR_BACKUP : DR_NONE, + txEntry.conflictExpireTime(), + near() ? null : txContext.explicitVer, + CU.subjectId(this, cctx), + resolveTaskName(), + txContext.dhtVer, + null); + } + + if (txContext.nearEntry != null) { + CacheObject valBytes = e.valueBytes(); + + txContext.nearEntry.updateOrEvict( + xidVer, + valBytes, + e.expireTime(), + e.ttl(), + nodeId, + topVer + ); + } + } + else if (txContext.operation == DELETE) { + result = e.innerRemove(this, + eventNodeId(), nodeId, - topVer); - - break; - } catch (GridCacheEntryRemovedException re) { - nearEntry = txContext.cacheContext.dht().near().peekExx(txEntry.key()); + false, + true, + true, + txEntry.keepBinary(), + txEntry.hasOldValue(), + txEntry.oldValue(), + topVer, + null, + txContext.cacheContext.isDrEnabled() ? DR_BACKUP : DR_NONE, + near() ? null : txContext.explicitVer, + CU.subjectId(this, cctx), + resolveTaskName(), + txContext.dhtVer, + txEntry.updateCounter() + ); + + // Keep near entry up to date. + if (txContext.nearEntry != null) + txContext.nearEntry.updateOrEvict(xidVer, null, 0, 0, nodeId, topVer); } - } - } - ); - } else if (txContext.operation == READ) { - assert near(); + else if (txContext.operation == RELOAD) { + CacheObject val = e.innerReload(); + + if (txContext.nearEntry != null) { + // Keep near entry up to date. + GridNearCacheEntry nearEntry = txContext.nearEntry; + + for (;;) { + try { + CacheObject reloaded = nearEntry.innerReload(); + + nearEntry.updateOrEvict(e.version(), + reloaded, + e.expireTime(), + e.ttl(), + nodeId, + topVer); + + break; + } + catch (GridCacheEntryRemovedException re) { + nearEntry = txContext.cacheContext.dht().near().peekExx(txEntry.key()); + } + } + } + + result = new GridCacheUpdateTxResult(true, val, null); + } + else if (txContext.operation == READ) { + assert near(); - operationFut = new CacheEntryOperationFuture<>(); + if (log.isDebugEnabled()) + log.debug("Ignoring READ entry when committing: " + txEntry); - operationFut.onDone(new GridCacheUpdateTxResult(false, null, null)); + result = new GridCacheUpdateTxResult(false, null, null); + } + // No-op. + else { + if (txContext.conflictContext == null || !txContext.conflictContext.isUseOld()) { + if (txEntry.ttl() != CU.TTL_NOT_CHANGED) + e.updateTtl(null, txEntry.ttl()); + } + + if (txContext.nearEntry != null) { + try { + CacheObject valBytes = e.valueBytes(); + + txContext.nearEntry.updateOrEvict(xidVer, + valBytes, + e.expireTime(), + e.ttl(), + nodeId, + topVer); + } + catch (GridCacheEntryRemovedException re) { + // This is impossible in TPP way, but anyway throw exception. + throw new IgniteCheckedException("Failed to get value bytes for entry: " + e, re); + } + } + + result = new GridCacheUpdateTxResult(true, null, null); + } - if (log.isDebugEnabled()) - log.debug("Ignoring READ entry when committing: " + txEntry); - } - // No-op. - else { - operationFut = cctx.cache().executor().execute(txContext.entry, e -> { - if (txContext.conflictContext == null || !txContext.conflictContext.isUseOld()) { - if (txEntry.ttl() != CU.TTL_NOT_CHANGED) - e.updateTtl(null, txEntry.ttl()); + // Assert after setting values as we want to make sure + // that if we replaced removed entries. + assert + txEntry.op() == READ || onePhaseCommit() || + // If candidate is not there, then lock was explicit + // and we simply allow the commit to proceed. + !txContext.entry.hasLockCandidateUnsafe(xidVer) || txContext.entry.lockedByUnsafe(xidVer) : + "Transaction does not own lock for commit [entry=" + txContext.entry + + ", tx=" + this + ']'; + + return result; } - - return new GridCacheUpdateTxResult(true, null, null); - }, - e -> refresh(txEntry), - (e, result) -> { - if (txContext.nearEntry == null) - return; - - try { - CacheObject valBytes = e.valueBytes(); - - txContext.nearEntry.updateOrEvict(xidVer, - valBytes, - e.expireTime(), - e.ttl(), - nodeId, - topVer); - } catch (GridCacheEntryRemovedException re) { - // This is impossible in TPP way, but anyway throw exception. - throw new IgniteCheckedException("Failed to get value bytes for entry: " + e, re); + catch (GridCacheEntryRemovedException re) { + txEntry.cached(refresh(txEntry)); } - }); - } + } + } + ); assert operationFut != null; commitEntriesFut.add(operationFut); - - // Assert after setting values as we want to make sure - // that if we replaced removed entries. - assert - txEntry.op() == READ || onePhaseCommit() || - // If candidate is not there, then lock was explicit - // and we simply allow the commit to proceed. - !txContext.entry.hasLockCandidateUnsafe(xidVer) || txContext.entry.lockedByUnsafe(xidVer) : - "Transaction does not own lock for commit [entry=" + txContext.entry + - ", tx=" + this + ']'; } } catch (Throwable t) { @@ -717,7 +709,7 @@ private boolean checkLocks() { if (log.isDebugEnabled()) log.debug("Got removed entry while committing (will retry): " + txEntry); - txEntry.cached(txEntry.context().cache().entryEx(txEntry.key(), topologyVersion())); + txEntry.cached(refresh(txEntry)); } } } @@ -1031,81 +1023,74 @@ private GridCacheEntryEx refresh(IgniteTxEntry txEntry) { return txEntry.context().cache().entryEx(txEntry.key(), topologyVersion()); } - private TransactionEntryContext prepareContext(IgniteTxEntry txEntry, GridCacheReturn ret) throws IgniteCheckedException { + private TransactionEntryContext prepareContext(IgniteTxEntry txEntry, GridCacheReturn ret) throws IgniteCheckedException, GridCacheEntryRemovedException { GridCacheContext cacheCtx = txEntry.context(); - for (;;) { - GridCacheEntryEx cached = txEntry.cached(); - - try { - if (cached == null) - txEntry.cached(cached = cacheCtx.cache().entryEx(txEntry.key(), topologyVersion())); - - if (near() && cacheCtx.dr().receiveEnabled()) { - cached.markObsolete(xidVer); - - return null; - } + GridCacheEntryEx cached = txEntry.cached(); - final GridNearCacheEntry nearCached = updateNearCache(cacheCtx, txEntry.key(), topVer) - ? cacheCtx.dht().near().peekExx(txEntry.key()) - : null; + if (cached == null) + txEntry.cached(cached = cacheCtx.cache().entryEx(txEntry.key(), topologyVersion())); - if (!F.isEmpty(txEntry.entryProcessors())) - txEntry.cached().unswap(false); + if (near() && cacheCtx.dr().receiveEnabled()) { + cached.markObsolete(xidVer); - IgniteBiTuple res = applyTransformClosures(txEntry, false, ret); + return null; + } - GridCacheOperation op = res.get1(); - CacheObject val = res.get2(); + final GridNearCacheEntry nearCached = updateNearCache(cacheCtx, txEntry.key(), topVer) + ? cacheCtx.dht().near().peekExx(txEntry.key()) + : null; - GridCacheVersion explicitVer = txEntry.conflictVersion(); + if (!F.isEmpty(txEntry.entryProcessors())) + txEntry.cached().unswap(false); - if (explicitVer == null) - explicitVer = writeVersion(); + IgniteBiTuple res = applyTransformClosures(txEntry, false, ret); - if (txEntry.ttl() == CU.TTL_ZERO) - op = DELETE; + GridCacheOperation op = res.get1(); + CacheObject val = res.get2(); - boolean conflictNeedResolve = cacheCtx.conflictNeedResolve(); + GridCacheVersion explicitVer = txEntry.conflictVersion(); - GridCacheVersionConflictContext conflictCtx = null; + if (explicitVer == null) + explicitVer = writeVersion(); - if (conflictNeedResolve) { - IgniteBiTuple - drRes = conflictResolve(op, txEntry, val, explicitVer, cached); + if (txEntry.ttl() == CU.TTL_ZERO) + op = DELETE; - assert drRes != null; + boolean conflictNeedResolve = cacheCtx.conflictNeedResolve(); - conflictCtx = drRes.get2(); + GridCacheVersionConflictContext conflictCtx = null; - if (conflictCtx.isUseOld()) - op = NOOP; - else if (conflictCtx.isUseNew()) { - txEntry.ttl(conflictCtx.ttl()); - txEntry.conflictExpireTime(conflictCtx.expireTime()); - } - else if (conflictCtx.isMerge()) { - op = drRes.get1(); - val = txEntry.context().toCacheObject(conflictCtx.mergeValue()); - explicitVer = writeVersion(); + if (conflictNeedResolve) { + IgniteBiTuple + drRes = conflictResolve(op, txEntry, val, explicitVer, cached); - txEntry.ttl(conflictCtx.ttl()); - txEntry.conflictExpireTime(conflictCtx.expireTime()); - } - } - else - // Nullify explicit version so that innerSet/innerRemove will work as usual. - explicitVer = null; + assert drRes != null; - GridCacheVersion dhtVer = cached.isNear() ? writeVersion() : null; + conflictCtx = drRes.get2(); - return new TransactionEntryContext(cacheCtx, op, dhtVer, explicitVer, val, cached, nearCached, conflictCtx); + if (conflictCtx.isUseOld()) + op = NOOP; + else if (conflictCtx.isUseNew()) { + txEntry.ttl(conflictCtx.ttl()); + txEntry.conflictExpireTime(conflictCtx.expireTime()); } - catch (GridCacheEntryRemovedException re) { - txEntry.cached(cacheCtx.cache().entryEx(txEntry.key(), topologyVersion())); + else if (conflictCtx.isMerge()) { + op = drRes.get1(); + val = txEntry.context().toCacheObject(conflictCtx.mergeValue()); + explicitVer = writeVersion(); + + txEntry.ttl(conflictCtx.ttl()); + txEntry.conflictExpireTime(conflictCtx.expireTime()); } } + else + // Nullify explicit version so that innerSet/innerRemove will work as usual. + explicitVer = null; + + GridCacheVersion dhtVer = cached.isNear() ? writeVersion() : null; + + return new TransactionEntryContext(cacheCtx, op, dhtVer, explicitVer, val, cached, nearCached, conflictCtx); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java index 4f1f17c0e0fb5..1f060f75e0f98 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java @@ -40,7 +40,6 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheEntryOperationFuture; import org.apache.ignite.internal.processors.cache.CacheEntryPredicate; -import org.apache.ignite.internal.processors.cache.CacheEntryRefresh; import org.apache.ignite.internal.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.CacheInvokeEntry; import org.apache.ignite.internal.processors.cache.CacheObject; @@ -64,7 +63,6 @@ import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException; import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException; import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException; -import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.lang.GridClosureException; import org.apache.ignite.internal.util.lang.GridTuple; @@ -479,6 +477,8 @@ protected GridCacheEntryEx entryEx(GridCacheContext cacheCtx, IgniteTxKey key, A @Override public IgniteTxCommitEntriesFuture startCommit() { TransactionState state = state(); + IgniteTxCommitEntriesFuture commitEntriesFut = new IgniteTxCommitEntriesFuture(); + try { if (state != COMMITTING) { if (remainingTime() == -1) @@ -503,7 +503,7 @@ protected GridCacheEntryEx entryEx(GridCacheContext cacheCtx, IgniteTxKey key, A cctx.tm().addCommittedTx(this); if (empty) - return IgniteTxCommitEntriesFuture.FINISHED; + return IgniteTxCommitEntriesFuture.EMPTY; batchStoreCommit(writeEntries()); @@ -512,376 +512,220 @@ protected GridCacheEntryEx entryEx(GridCacheContext cacheCtx, IgniteTxKey key, A try { cctx.tm().txContext(this); - IgniteTxCommitEntriesFuture commitEntriesFuture = new IgniteTxCommitEntriesFuture(); - /* * Commit to cache. Note that for 'near' transaction we loop through all the entries. */ for (final IgniteTxEntry txEntry : commitEntries) { - GridCacheContext cacheCtx = txEntry.context(); - - GridDrType drType = cacheCtx.isDrEnabled() ? DR_PRIMARY : DR_NONE; - - UUID nodeId = txEntry.nodeId() == null ? this.nodeId : txEntry.nodeId(); - - CacheEntryRefresh refresh = entry -> entryEx(cacheCtx, txEntry.txKey(), topologyVersion()); - - try { - GridCacheEntryEx cached = txEntry.cached(); - - // Must try to evict near entries before committing from - // transaction manager to make sure locks are held. - if (evictNearEntry(txEntry, false)) { - commitEntriesFuture.add(new GridFinishedFuture<>()); - - continue; - } - - if (cacheCtx.isNear() && cacheCtx.dr().receiveEnabled()) { - cached.markObsolete(xidVer); - - commitEntriesFuture.add(new GridFinishedFuture<>()); - - continue; - } - - if (cached.detached()) { - commitEntriesFuture.add(new GridFinishedFuture<>()); - - continue; - } - - final boolean updateNearCache = updateNearCache(cacheCtx, txEntry.key(), topVer); - - final boolean updateMetrics = updateNearCache || !cacheCtx.isNear() || !txEntry.locallyMapped(); - - final boolean fireEvent = !isNearLocallyMapped(txEntry, false); - - if (!F.isEmpty(txEntry.entryProcessors()) || !F.isEmpty(txEntry.filters())) - txEntry.cached().unswap(false); - - IgniteBiTuple res = applyTransformClosures(txEntry, - true, null); - - GridCacheVersion dhtVer = null; - - // For near local transactions we must record DHT version - // in order to keep near entries on backup nodes until - // backup remote transaction completes. - if (cacheCtx.isNear()) { - if (txEntry.op() == CREATE || txEntry.op() == UPDATE || - txEntry.op() == DELETE || txEntry.op() == TRANSFORM) - dhtVer = txEntry.dhtVersion(); - - if ((txEntry.op() == CREATE || txEntry.op() == UPDATE) && - txEntry.conflictExpireTime() == CU.EXPIRE_TIME_CALCULATE) { - ExpiryPolicy expiry = cacheCtx.expiryForTxEntry(txEntry); - - if (expiry != null) { - txEntry.cached().unswap(false); - - Duration duration = cached.hasValue() ? - expiry.getExpiryForUpdate() : expiry.getExpiryForCreation(); - - txEntry.ttl(CU.toTtl(duration)); - } - } - } - - GridCacheOperation op = res.get1(); - CacheObject val = res.get2(); - - // Deal with conflicts. - GridCacheVersion explicitVer = txEntry.conflictVersion() != null ? - txEntry.conflictVersion() : writeVersion(); - - if ((op == CREATE || op == UPDATE) && - txEntry.conflictExpireTime() == CU.EXPIRE_TIME_CALCULATE) { - ExpiryPolicy expiry = cacheCtx.expiryForTxEntry(txEntry); - - if (expiry != null) { - Duration duration = cached.hasValue() ? - expiry.getExpiryForUpdate() : expiry.getExpiryForCreation(); - - long ttl = CU.toTtl(duration); - - txEntry.ttl(ttl); - - if (ttl == CU.TTL_ZERO) - op = DELETE; - } - } - - boolean conflictNeedResolve = cacheCtx.conflictNeedResolve(); - - GridCacheVersionConflictContext conflictCtx = null; - - if (conflictNeedResolve) { - IgniteBiTuple conflictRes = - conflictResolve(op, txEntry, val, explicitVer, cached); + final UUID nodeId = txEntry.nodeId() == null ? this.nodeId : txEntry.nodeId(); + final GridDrType drType = txEntry.context().isDrEnabled() ? DR_PRIMARY : DR_NONE; - assert conflictRes != null; + CacheEntryOperationFuture operationFut = cctx.cache().executor().execute( + txEntry.key().partition(), + () -> { + for (;;) { + try { + TransactionEntryContext txContext = prepareContext(txEntry); - conflictCtx = conflictRes.get2(); + if (txContext == null) + return new GridCacheUpdateTxResult(false, null, null); - if (conflictCtx.isUseOld()) - op = NOOP; - else if (conflictCtx.isUseNew()) { - txEntry.ttl(conflictCtx.ttl()); - txEntry.conflictExpireTime(conflictCtx.expireTime()); - } - else { - assert conflictCtx.isMerge(); + GridCacheUpdateTxResult result; - op = conflictRes.get1(); - val = txEntry.context().toCacheObject(conflictCtx.mergeValue()); - explicitVer = writeVersion(); + GridCacheEntryEx e = txContext.entry; - txEntry.ttl(conflictCtx.ttl()); - txEntry.conflictExpireTime(conflictCtx.expireTime()); - } - } - else - // Nullify explicit version so that innerSet/innerRemove will work as usual. - explicitVer = null; + if (txContext.operation == CREATE || txContext.operation == UPDATE) { + assert txContext.value != null : txEntry; - if (sndTransformedVals || conflictNeedResolve) { - assert sndTransformedVals && cacheCtx.isReplicated() || conflictNeedResolve; - - txEntry.value(val, true, false); - txEntry.op(op); - txEntry.entryProcessors(null); - txEntry.conflictVersion(explicitVer); - } - - if (dhtVer == null) - dhtVer = explicitVer != null ? explicitVer : writeVersion(); - - final GridCacheVersion explicitVer0 = explicitVer; - final GridCacheVersion dhtVer0 = dhtVer; - final CacheObject val0 = val; - - CacheEntryOperationFuture operationFuture; - - if (op == CREATE || op == UPDATE) { - assert val != null : txEntry; - - operationFuture = cctx.cache().executor().execute(cached, e -> e.innerSet( - this, - eventNodeId(), - txEntry.nodeId(), - val0, - false, - false, - txEntry.ttl(), - fireEvent, - updateMetrics, - txEntry.keepBinary(), - txEntry.hasOldValue(), - txEntry.oldValue(), - topVer, - null, - cached.detached() ? DR_NONE : drType, - txEntry.conflictExpireTime(), - cached.isNear() ? null : explicitVer0, - CU.subjectId(this, cctx), - resolveTaskName(), - dhtVer0, - null), - refresh, - ((entry, result) -> { - if (updateNearCache && result.success()) { - updateNearEntrySafely(cacheCtx, txEntry.key(), nearEntry -> nearEntry.innerSet( - null, + result = e.innerSet( + this, eventNodeId(), - nodeId, - val0, + txEntry.nodeId(), + txContext.value, false, false, txEntry.ttl(), - false, - updateMetrics, + txContext.fireEvent, + txContext.updateMetrics, txEntry.keepBinary(), txEntry.hasOldValue(), txEntry.oldValue(), topVer, - CU.empty0(), - DR_NONE, - txEntry.conflictExpireTime(), null, + e.detached() ? DR_NONE : drType, + txEntry.conflictExpireTime(), + e.isNear() ? null : txContext.explicitVer, CU.subjectId(this, cctx), resolveTaskName(), - dhtVer0, - null) + txContext.dhtVer, + null ); + + if (txContext.updateNearCache && result.success()) { + updateNearEntrySafely(txContext.cacheContext, txEntry.key(), nearEntry -> nearEntry.innerSet( + null, + eventNodeId(), + nodeId, + txContext.value, + false, + false, + txEntry.ttl(), + false, + txContext.updateMetrics, + txEntry.keepBinary(), + txEntry.hasOldValue(), + txEntry.oldValue(), + topVer, + CU.empty0(), + DR_NONE, + txEntry.conflictExpireTime(), + null, + CU.subjectId(this, cctx), + resolveTaskName(), + txContext.dhtVer, + null) + ); + } } - }) - ); - } - else if (op == DELETE) { - operationFuture = cctx.cache().executor().execute(cached, e -> e.innerRemove( - this, - eventNodeId(), - txEntry.nodeId(), - false, - fireEvent, - updateMetrics, - txEntry.keepBinary(), - txEntry.hasOldValue(), - txEntry.oldValue(), - topVer, - null, - cached.detached() ? DR_NONE : drType, - cached.isNear() ? null : explicitVer0, - CU.subjectId(this, cctx), - resolveTaskName(), - dhtVer0, - null), - refresh, - ((entry, result) -> { - if (updateNearCache && result.success()) { - updateNearEntrySafely(cacheCtx, txEntry.key(), nearEntry -> nearEntry.innerRemove( - null, + else if (txContext.operation == DELETE) { + result = e.innerRemove( + this, eventNodeId(), - nodeId, - false, + txEntry.nodeId(), false, - updateMetrics, + txContext.fireEvent, + txContext.updateMetrics, txEntry.keepBinary(), txEntry.hasOldValue(), txEntry.oldValue(), topVer, - CU.empty0(), - DR_NONE, null, + e.detached() ? DR_NONE : drType, + e.isNear() ? null : txContext.explicitVer, CU.subjectId(this, cctx), resolveTaskName(), - dhtVer0, - null) - ); - } - - // Check commit locks after set, to make sure that - // we are not changing obsolete entries. - // (innerSet and innerRemove will throw an exception - // if an entry is obsolete). - if (txEntry.op() != READ) - checkCommitLocks(entry); - }) - ); - } - else if (op == RELOAD) { - operationFuture = cctx.cache().executor().execute( - cached, e -> { - e.innerReload(); - - return new GridCacheUpdateTxResult( - true, - null, - null - ); - }, - refresh, - (entry, result) -> { - if (updateNearCache && result.success()) - updateNearEntrySafely(cacheCtx, txEntry.key(), nearEntry -> nearEntry.innerReload()); - } - ); - } - else if (op == READ) { - operationFuture = cctx.cache().executor().execute( - cached, e -> { - CacheGroupContext grp = cacheCtx.group(); - WALPointer ptr0 = null; - - if (grp.persistenceEnabled() && grp.walEnabled() && - cctx.snapshot().needTxReadLogging()) { - ptr0 = cctx.wal().log(new DataRecord(new DataEntry( - cacheCtx.cacheId(), - txEntry.key(), - val0, - READ, - nearXidVersion(), - writeVersion(), - 0, - txEntry.key().partition(), - txEntry.updateCounter()))); + txContext.dhtVer, + null); + + if (txContext.updateNearCache && result.success()) { + updateNearEntrySafely(txContext.cacheContext, txEntry.key(), nearEntry -> nearEntry.innerRemove( + null, + eventNodeId(), + nodeId, + false, + false, + txContext.updateMetrics, + txEntry.keepBinary(), + txEntry.hasOldValue(), + txEntry.oldValue(), + topVer, + CU.empty0(), + DR_NONE, + null, + CU.subjectId(this, cctx), + resolveTaskName(), + txContext.dhtVer, + null) + ); + } } + else if (txContext.operation == RELOAD) { + e.innerReload(); - ExpiryPolicy expiry = cacheCtx.expiryForTxEntry(txEntry); - - if (expiry != null) { - Duration duration = expiry.getExpiryForAccess(); + result = new GridCacheUpdateTxResult(true, null, null); - if (duration != null) - cached.updateTtl(null, CU.toTtl(duration)); + if (txContext.updateNearCache && result.success()) + updateNearEntrySafely(txContext.cacheContext, txEntry.key(), nearEntry -> nearEntry.innerReload()); } + else if (txContext.operation == READ) { + CacheGroupContext grp = txContext.cacheContext.group(); + WALPointer ptr = null; + + if (grp.persistenceEnabled() && grp.walEnabled() && + cctx.snapshot().needTxReadLogging()) { + ptr = cctx.wal().log(new DataRecord(new DataEntry( + txContext.cacheContext.cacheId(), + txEntry.key(), + txContext.value, + READ, + nearXidVersion(), + writeVersion(), + 0, + txEntry.key().partition(), + txEntry.updateCounter()))); + } + + ExpiryPolicy expiry = txContext.cacheContext.expiryForTxEntry(txEntry); + + if (expiry != null) { + Duration duration = expiry.getExpiryForAccess(); + + if (duration != null) + e.updateTtl(null, CU.toTtl(duration)); + } + + if (log.isDebugEnabled()) + log.debug("Ignoring READ entry when committing: " + txEntry); + + result = new GridCacheUpdateTxResult(true, txContext.value, ptr); + } + else { + assert ownsLock(txEntry.cached()): + "Transaction does not own lock for group lock entry during commit [tx=" + + this + ", txEntry=" + txEntry + ']'; - if (log.isDebugEnabled()) - log.debug("Ignoring READ entry when committing: " + txEntry); - - return new GridCacheUpdateTxResult( - true, - val0, - ptr0 - ); - }, - refresh, - (entry, result) -> { } - ); - } - else { - GridCacheVersionConflictContext conflictCtx0 = conflictCtx; + if (txContext.conflictContext == null || !txContext.conflictContext.isUseOld()) { + if (txEntry.ttl() != CU.TTL_NOT_CHANGED) + e.updateTtl(null, txEntry.ttl()); + } - operationFuture = cctx.cache().executor().execute( - cached, e -> { - assert ownsLock(txEntry.cached()): - "Transaction does not own lock for group lock entry during commit [tx=" + - this + ", txEntry=" + txEntry + ']'; + if (log.isDebugEnabled()) + log.debug("Ignoring NOOP entry when committing: " + txEntry); - if (conflictCtx0 == null || !conflictCtx0.isUseOld()) { - if (txEntry.ttl() != CU.TTL_NOT_CHANGED) - cached.updateTtl(null, txEntry.ttl()); + result = new GridCacheUpdateTxResult(false, txContext.value, null); } - if (log.isDebugEnabled()) - log.debug("Ignoring NOOP entry when committing: " + txEntry); - - return new GridCacheUpdateTxResult( - false, - val0, - null - ); - }, - refresh, - (entry, result) -> { } - ); - } + // Check commit locks after set, to make sure that + // we are not changing obsolete entries. + // (innerSet and innerRemove will throw an exception + // if an entry is obsolete). + if (txEntry.op() != READ) + checkCommitLocks(e); - assert operationFuture != null : "Entry operation must be submitted"; + return result; + } + catch (GridCacheEntryRemovedException re) { + txEntry.cached(refresh(txEntry)); + } + } + } + ); - commitEntriesFuture.add(operationFuture); - } - catch (Throwable ex) { - commitEntriesFuture.onDone(ex); + assert operationFut != null : "Entry operation must be submitted"; - break; - } + commitEntriesFut.add(operationFut); } - commitEntriesFuture.markInitialized(); + commitEntriesFut.markInitialized(); - return commitEntriesFuture; + return commitEntriesFut; } finally { cctx.tm().resetContext(); } } catch (Throwable t) { - return IgniteTxCommitEntriesFuture.finishedWithError(t); + commitEntriesFut.markInitialized(); + + commitEntriesFut.onDone(t); + + return commitEntriesFut; } } + private GridCacheEntryEx refresh(IgniteTxEntry txEntry) { + return txEntry.context().cache().entryEx(txEntry.key(), topologyVersion()); + } + private Collection commitEntries() { return (near() || cctx.snapshot().needTxReadLogging()) ? allEntries() : writeEntries(); } @@ -1584,6 +1428,164 @@ else if (explicitCand.dhtLocal()) } } + /** + * Context to perform operation with {@link IgniteTxEntry}. + */ + class TransactionEntryContext { + private final GridCacheContext cacheContext; + private final GridCacheOperation operation; + private final GridCacheVersion dhtVer; + private final GridCacheVersion explicitVer; + private final CacheObject value; + private final GridCacheEntryEx entry; + private final boolean updateNearCache; + private final GridCacheVersionConflictContext conflictContext; + private final boolean updateMetrics; + private final boolean fireEvent; + + public TransactionEntryContext(GridCacheContext cacheContext, GridCacheOperation operation, GridCacheVersion dhtVer, GridCacheVersion explicitVer, CacheObject value, GridCacheEntryEx entry, boolean updateNearCache, GridCacheVersionConflictContext conflictContext, boolean updateMetrics, boolean fireEvent) { + this.cacheContext = cacheContext; + this.operation = operation; + this.dhtVer = dhtVer; + this.explicitVer = explicitVer; + this.value = value; + this.entry = entry; + this.updateNearCache = updateNearCache; + this.conflictContext = conflictContext; + this.updateMetrics = updateMetrics; + this.fireEvent = fireEvent; + } + } + + + private TransactionEntryContext prepareContext(IgniteTxEntry txEntry) throws IgniteCheckedException, GridCacheEntryRemovedException { + GridCacheContext cacheCtx = txEntry.context(); + + GridCacheEntryEx cached = txEntry.cached(); + + // Must try to evict near entries before committing from + // transaction manager to make sure locks are held. + if (evictNearEntry(txEntry, false)) + return null; + + if (cacheCtx.isNear() && cacheCtx.dr().receiveEnabled()) { + cached.markObsolete(xidVer); + + return null; + } + + if (cached.detached()) + return null; + + final boolean updateNearCache = updateNearCache(cacheCtx, txEntry.key(), topVer); + + final boolean updateMetrics = updateNearCache || !cacheCtx.isNear() || !txEntry.locallyMapped(); + + final boolean fireEvent = !isNearLocallyMapped(txEntry, false); + + if (!F.isEmpty(txEntry.entryProcessors()) || !F.isEmpty(txEntry.filters())) + txEntry.cached().unswap(false); + + IgniteBiTuple res = applyTransformClosures(txEntry, + true, null); + + GridCacheVersion dhtVer = null; + + // For near local transactions we must record DHT version + // in order to keep near entries on backup nodes until + // backup remote transaction completes. + if (cacheCtx.isNear()) { + if (txEntry.op() == CREATE || txEntry.op() == UPDATE || + txEntry.op() == DELETE || txEntry.op() == TRANSFORM) + dhtVer = txEntry.dhtVersion(); + + if ((txEntry.op() == CREATE || txEntry.op() == UPDATE) && + txEntry.conflictExpireTime() == CU.EXPIRE_TIME_CALCULATE) { + ExpiryPolicy expiry = cacheCtx.expiryForTxEntry(txEntry); + + if (expiry != null) { + txEntry.cached().unswap(false); + + Duration duration = cached.hasValue() ? + expiry.getExpiryForUpdate() : expiry.getExpiryForCreation(); + + txEntry.ttl(CU.toTtl(duration)); + } + } + } + + GridCacheOperation op = res.get1(); + CacheObject val = res.get2(); + + // Deal with conflicts. + GridCacheVersion explicitVer = txEntry.conflictVersion() != null ? + txEntry.conflictVersion() : writeVersion(); + + if ((op == CREATE || op == UPDATE) && + txEntry.conflictExpireTime() == CU.EXPIRE_TIME_CALCULATE) { + ExpiryPolicy expiry = cacheCtx.expiryForTxEntry(txEntry); + + if (expiry != null) { + Duration duration = cached.hasValue() ? + expiry.getExpiryForUpdate() : expiry.getExpiryForCreation(); + + long ttl = CU.toTtl(duration); + + txEntry.ttl(ttl); + + if (ttl == CU.TTL_ZERO) + op = DELETE; + } + } + + boolean conflictNeedResolve = cacheCtx.conflictNeedResolve(); + + GridCacheVersionConflictContext conflictCtx = null; + + if (conflictNeedResolve) { + IgniteBiTuple conflictRes = + conflictResolve(op, txEntry, val, explicitVer, cached); + + assert conflictRes != null; + + conflictCtx = conflictRes.get2(); + + if (conflictCtx.isUseOld()) + op = NOOP; + else if (conflictCtx.isUseNew()) { + txEntry.ttl(conflictCtx.ttl()); + txEntry.conflictExpireTime(conflictCtx.expireTime()); + } + else { + assert conflictCtx.isMerge(); + + op = conflictRes.get1(); + val = txEntry.context().toCacheObject(conflictCtx.mergeValue()); + explicitVer = writeVersion(); + + txEntry.ttl(conflictCtx.ttl()); + txEntry.conflictExpireTime(conflictCtx.expireTime()); + } + } + else + // Nullify explicit version so that innerSet/innerRemove will work as usual. + explicitVer = null; + + if (sndTransformedVals || conflictNeedResolve) { + assert sndTransformedVals && cacheCtx.isReplicated() || conflictNeedResolve; + + txEntry.value(val, true, false); + txEntry.op(op); + txEntry.entryProcessors(null); + txEntry.conflictVersion(explicitVer); + } + + if (dhtVer == null) + dhtVer = explicitVer != null ? explicitVer : writeVersion(); + + return new TransactionEntryContext(cacheCtx, op, dhtVer, explicitVer, val, cached, updateNearCache, conflictCtx, updateMetrics, fireEvent); + } + /** {@inheritDoc} */ @Override public String toString() { return GridToStringBuilder.toString(IgniteTxLocalAdapter.class, this, "super", super.toString(), From f29df886271bc63a20c8215021f05429c3c82d5c Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Thu, 30 Aug 2018 17:16:24 +0300 Subject: [PATCH 17/18] IGNITE-9270 Parallelize tx finishing operations. --- .../GridDistributedTxRemoteAdapter.java | 26 +++--- .../cache/transactions/IgniteTxManager.java | 80 +++++++++++++------ 2 files changed, 70 insertions(+), 36 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java index c19cee6fccf8b..87c16a60e242f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java @@ -331,21 +331,23 @@ private void doneRemote(IgniteTxEntry txEntry, if (!hasWriteKey(entry.txKey())) return false; - IgniteTxCommitEntriesFuture commitFut = startCommitEntries(); + cctx.tm().finisher().execute(this, () -> { + IgniteTxCommitEntriesFuture commitFut = startCommitEntries(); - commitFut.listen(f -> cctx.tm().finisher().execute(this, () -> { - try { - finishCommitEntries(commitFut); - } - catch (IgniteCheckedException e) { - U.error(log, "Failed to commit remote transaction: " + this, e); + commitFut.listen(f -> cctx.tm().finisher().execute(this, () -> { + try { + finishCommitEntries(commitFut); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to commit remote transaction: " + this, e); - invalidate(true); - systemInvalidate(true); + invalidate(true); + systemInvalidate(true); - rollbackRemoteTx(); - } - })); + rollbackRemoteTx(); + } + })); + }); return true; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java index 030fbb7c63c58..95ff9889ce8e8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java @@ -177,12 +177,30 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter { /** Committed local transactions. */ private final ConcurrentLinkedHashMap completedVersHashMap = - new ConcurrentLinkedHashMap<>( + new ConcurrentLinkedHashMap( Integer.getInteger(IGNITE_MAX_COMPLETED_TX_COUNT, DFLT_MAX_COMPLETED_TX_CNT), 0.75f, Runtime.getRuntime().availableProcessors() * 2, Integer.getInteger(IGNITE_MAX_COMPLETED_TX_COUNT, DFLT_MAX_COMPLETED_TX_CNT), - PER_SEGMENT_Q); + PER_SEGMENT_Q) { +/* + + @Override public Object putIfAbsent(Object key, Object val) { + System.err.println("[PUT] " + Thread.currentThread().getName()); + + if (!Thread.currentThread().getName().contains("dedicated")) + U.dumpStack(log, "[PUT] to completed vers hashmap."); + + return super.putIfAbsent(key, val); + } + + @Override public Object get(Object key) { + System.err.println("[GET] " + Thread.currentThread().getName()); + + return super.get(key); + } +*/ + }; /** Pending one phase commit ack requests sender. */ private GridDeferredAckMessageSender deferredAckMsgSnd; @@ -910,32 +928,36 @@ private void removeObsolete(IgniteInternalTx tx) { Collection entries = tx.local() ? tx.allEntries() : tx.writeEntries(); for (IgniteTxEntry entry : entries) { - GridCacheEntryEx cached = entry.cached(); + cctx.cache().executor().execute(entry.key().partition(), () -> { + GridCacheEntryEx cached = entry.cached(); - GridCacheContext cacheCtx = entry.context(); + GridCacheContext cacheCtx = entry.context(); - if (cached == null) - cached = cacheCtx.cache().peekEx(entry.key()); + if (cached == null) + cached = cacheCtx.cache().peekEx(entry.key()); - if (cached.detached()) - continue; + if (cached.detached()) + return null; - try { - if (cached.obsolete() || cached.markObsoleteIfEmpty(tx.xidVersion())) - cacheCtx.cache().removeEntry(cached); + try { + if (cached.obsolete() || cached.markObsoleteIfEmpty(tx.xidVersion())) + cacheCtx.cache().removeEntry(cached); - if (!tx.near() && isNearEnabled(cacheCtx)) { - GridNearCacheAdapter near = cacheCtx.isNear() ? cacheCtx.near() : cacheCtx.dht().near(); + if (!tx.near() && isNearEnabled(cacheCtx)) { + GridNearCacheAdapter near = cacheCtx.isNear() ? cacheCtx.near() : cacheCtx.dht().near(); - GridNearCacheEntry e = near.peekExx(entry.key()); + GridNearCacheEntry e = near.peekExx(entry.key()); - if (e != null && e.markObsoleteIfEmpty(null)) - near.removeEntry(e); + if (e != null && e.markObsoleteIfEmpty(null)) + near.removeEntry(e); + } } - } - catch (IgniteCheckedException e) { - U.error(log, "Failed to remove obsolete entry from cache: " + cached, e); - } + catch (IgniteCheckedException e) { + U.error(log, "Failed to remove obsolete entry from cache: " + cached, e); + } + + return null; + }); } } @@ -1527,8 +1549,13 @@ private void notifyEvictions(IgniteInternalTx tx) { if (tx.internal()) return; - for (IgniteTxEntry txEntry : tx.allEntries()) - txEntry.cached().context().evicts().touch(txEntry, tx.local()); + for (IgniteTxEntry txEntry : tx.allEntries()) { + cctx.cache().executor().execute(txEntry.key().partition(), () -> { + txEntry.cached().context().evicts().touch(txEntry, tx.local()); + + return null; + }); + } } /** @@ -1745,8 +1772,13 @@ private void txUnlock(IgniteInternalTx tx, IgniteTxEntry txEntry) { * @param entries Entries to unlock. */ private void unlockMultiple(IgniteInternalTx tx, Iterable entries) { - for (IgniteTxEntry txEntry : entries) - txUnlock(tx, txEntry); + for (IgniteTxEntry txEntry : entries) { + cctx.cache().executor().execute(txEntry.key().partition(), () -> { + txUnlock(tx, txEntry); + + return null; + }); + } } /** From 83ff6363259fb0fe884973c73fd977b27e26b259 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Thu, 30 Aug 2018 17:58:17 +0300 Subject: [PATCH 18/18] IGNITE-9270 Dedicated worker polls forever. --- .../ignite/internal/util/StripedExecutor.java | 32 +++++++++++++++---- 1 file changed, 26 insertions(+), 6 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java b/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java index 4d41040e7bec3..464380a8ab4d7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java @@ -120,12 +120,12 @@ public StripedExecutor( try { for (int i = 0; i < cnt; i++) { stripes[i] = stealTasks - ? new StripeConcurrentQueue(igniteInstanceName, poolName, i, log, stripes, errHnd, gridWorkerLsnr) - : new StripeConcurrentQueue(igniteInstanceName, poolName, i, log, errHnd, gridWorkerLsnr); + ? new StripeConcurrentQueue(igniteInstanceName, poolName, i, log, stripes, errHnd, gridWorkerLsnr, false) + : new StripeConcurrentQueue(igniteInstanceName, poolName, i, log, errHnd, gridWorkerLsnr, false); } for (int i = cnt; i < cnt + dedicated; i++) - stripes[i] = new StripeConcurrentQueue(igniteInstanceName, poolName + "-dedicated", i, log, errHnd, gridWorkerLsnr); + stripes[i] = new StripeConcurrentQueue(igniteInstanceName, poolName + "-dedicated", i, log, errHnd, gridWorkerLsnr, true); for (int i = 0; i < cnt + dedicated; i++) stripes[i].start(); @@ -582,6 +582,9 @@ private static class StripeConcurrentQueue extends Stripe { /** */ private volatile boolean parked; + /** */ + private final boolean dedicated; + /** * @param igniteInstanceName Ignite instance name. * @param poolName Pool name. @@ -596,9 +599,10 @@ private static class StripeConcurrentQueue extends Stripe { int idx, IgniteLogger log, IgniteInClosure errHnd, - GridWorkerListener gridWorkerLsnr + GridWorkerListener gridWorkerLsnr, + boolean dedicated ) { - this(igniteInstanceName, poolName, idx, log, null, errHnd, gridWorkerLsnr); + this(igniteInstanceName, poolName, idx, log, null, errHnd, gridWorkerLsnr, dedicated); } /** @@ -616,7 +620,8 @@ private static class StripeConcurrentQueue extends Stripe { IgniteLogger log, Stripe[] others, IgniteInClosure errHnd, - GridWorkerListener gridWorkerLsnr + GridWorkerListener gridWorkerLsnr, + boolean dedicated ) { super( igniteInstanceName, @@ -629,12 +634,27 @@ private static class StripeConcurrentQueue extends Stripe { this.others = others; this.queue = others == null ? new ConcurrentLinkedQueue() : new ConcurrentLinkedDeque(); + + this.dedicated = dedicated; } /** {@inheritDoc} */ @Override Runnable take() throws InterruptedException { Runnable r; + if (dedicated) { + for (int i = 0; ; i++) { + r = queue.poll(); + + if (r != null) + return r; + + if (i % 2048 == 0) + if (Thread.interrupted()) + throw new InterruptedException(); + } + } + for (int i = 0; i < 2048; i++) { r = queue.poll();