You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2017/03/02 15:17:23 UTC

ignite git commit: ignite-4768

Repository: ignite
Updated Branches:
  refs/heads/ignite-4768 [created] d39050c74


ignite-4768


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/d39050c7
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/d39050c7
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/d39050c7

Branch: refs/heads/ignite-4768
Commit: d39050c749c6ec2876058d80ddb591d9b5cdbdcf
Parents: 50f8741
Author: sboikov <sb...@gridgain.com>
Authored: Thu Mar 2 14:32:16 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Mar 2 15:08:17 2017 +0300

----------------------------------------------------------------------
 .../distributed/GridDistributedBaseMessage.java |  2 +-
 ...arOptimisticSerializableTxPrepareFuture.java | 10 ---
 .../near/GridNearTxFinishFuture.java            | 72 ++++----------------
 .../cache/transactions/IgniteTxHandler.java     |  4 +-
 4 files changed, 17 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d39050c7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedBaseMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedBaseMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedBaseMessage.java
index ebbc9ae..630c79f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedBaseMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedBaseMessage.java
@@ -142,7 +142,7 @@ public abstract class GridDistributedBaseMessage extends GridCacheMessage implem
     /**
      * @return Count of keys referenced in candidates array (needed only locally for optimization).
      */
-    public int keysCount() {
+    int keysCount() {
         return cnt;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d39050c7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
index dbc8096..e450d4b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
@@ -70,9 +70,6 @@ import static org.apache.ignite.transactions.TransactionState.PREPARING;
  */
 public class GridNearOptimisticSerializableTxPrepareFuture extends GridNearOptimisticTxPrepareFutureAdapter {
     /** */
-    public static final IgniteProductVersion SER_TX_SINCE = IgniteProductVersion.fromString("1.5.0");
-
-    /** */
     @GridToStringExclude
     private KeyLockFuture keyLockFut;
 
@@ -544,13 +541,6 @@ public class GridNearOptimisticSerializableTxPrepareFuture extends GridNearOptim
                 ", primary=" + U.toShortString(primary) + ", topVer=" + topVer + ']');
         }
 
-        if (primary.version().compareTo(SER_TX_SINCE) < 0) {
-            onDone(new IgniteCheckedException("Optimistic serializable transactions can be used only with node " +
-                "version starting from " + SER_TX_SINCE));
-
-            return;
-        }
-
         // Must re-initialize cached entry while holding topology lock.
         if (cacheCtx.isNear())
             entry.cached(cacheCtx.nearTx().entryExx(entry.key(), topVer));

http://git-wip-us.apache.org/repos/asf/ignite/blob/d39050c7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
----------------------------------------------------------------------
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 512f63e..b7fdba3 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
@@ -54,12 +54,11 @@ 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.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.transactions.TransactionRollbackException;
 
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.NOOP;
 import static org.apache.ignite.transactions.TransactionState.UNKNOWN;
 
@@ -69,18 +68,6 @@ import static org.apache.ignite.transactions.TransactionState.UNKNOWN;
 public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFuture<IgniteInternalTx>
     implements GridCacheFuture<IgniteInternalTx> {
     /** */
-    public static final IgniteProductVersion FINISH_NEAR_ONE_PHASE_SINCE = IgniteProductVersion.fromString("1.4.0");
-
-    /** */
-    public static final IgniteProductVersion WAIT_REMOTE_TXS_SINCE = IgniteProductVersion.fromString("1.5.1");
-
-    /** */
-    public static final IgniteProductVersion PRIMARY_SYNC_TXS_SINCE = IgniteProductVersion.fromString("1.6.0");
-
-    /** */
-    public static final IgniteProductVersion ACK_DHT_ONE_PHASE_SINCE = IgniteProductVersion.fromString("1.6.8");
-
-    /** */
     private static final long serialVersionUID = 0L;
 
     /** Logger reference. */
@@ -157,7 +144,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
     @Override public boolean onNodeLeft(UUID nodeId) {
         boolean found = false;
 
-        for (IgniteInternalFuture<?> fut : futures())
+        for (IgniteInternalFuture<?> fut : futures()) {
             if (isMini(fut)) {
                 MinFuture f = (MinFuture)fut;
 
@@ -168,6 +155,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
                     found = true;
                 }
             }
+        }
 
         return found;
     }
@@ -470,10 +458,8 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
                 }
                 else if (backup.isLocal())
                     cctx.tm().removeTxReturn(tx.xidVersion());
-                else {
-                    if (ACK_DHT_ONE_PHASE_SINCE.compareToIgnoreTimestamp(backup.version()) <= 0)
-                        cctx.tm().sendDeferredAckResponse(backupId, tx.xidVersion());
-                }
+                else
+                    cctx.tm().sendDeferredAckResponse(backupId, tx.xidVersion());
             }
         }
     }
@@ -575,24 +561,13 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
                     else {
                         GridDhtTxFinishRequest finishReq = checkCommittedRequest(mini.futureId(), false);
 
-                        // Preserve old behavior, otherwise response is not sent.
-                        if (WAIT_REMOTE_TXS_SINCE.compareTo(backup.version()) > 0)
-                            finishReq.syncCommit(true);
-
                         try {
-                            if (FINISH_NEAR_ONE_PHASE_SINCE.compareTo(backup.version()) <= 0) {
-                                cctx.io().send(backup, finishReq, tx.ioPolicy());
+                            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() + ']');
-                                }
-                            }
-                            else {
-                                mini.onDone(new IgniteTxHeuristicCheckedException("Failed to check for tx commit on " +
-                                    "the backup node (node has an old Ignite version) [rmtNodeId=" + backup.id() +
-                                    ", ver=" + backup.version() + ']'));
+                            if (msgLog.isDebugEnabled()) {
+                                msgLog.debug("Near finish fut, sent check committed request [" +
+                                    "txId=" + tx.nearXidVersion() +
+                                    ", node=" + backup.id() + ']');
                             }
                         }
                         catch (ClusterTopologyCheckedException ignored) {
@@ -624,18 +599,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
         if (tx.mappings().empty())
             return false;
 
-        boolean finish = tx.txState().hasNearCache(cctx) || !commit;
-
-        if (finish) {
-            GridDistributedTxMapping mapping = tx.mappings().singleMapping();
-
-            assert mapping != null : tx;
-
-            if (FINISH_NEAR_ONE_PHASE_SINCE.compareTo(mapping.node().version()) > 0)
-                finish = false;
-        }
-
-        return finish;
+        return tx.txState().hasNearCache(cctx) || !commit;
     }
 
     /**
@@ -755,12 +719,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
                         ", node=" + n.id() + ']');
                 }
 
-                boolean wait;
-
-                if (syncMode == PRIMARY_SYNC)
-                    wait = n.version().compareToIgnoreTimestamp(PRIMARY_SYNC_TXS_SINCE) >= 0;
-                else
-                    wait = syncMode == FULL_SYNC;
+                boolean wait = syncMode != FULL_ASYNC;
 
                 // If we don't wait for result, then mark future as done.
                 if (!wait)
@@ -893,8 +852,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
     }
 
     /**
-     * Mini-future for get operations. Mini-futures are only waiting on a single
-     * node as opposed to multiple nodes.
+     *
      */
     private class FinishMiniFuture extends MinFuture {
         /** */
@@ -949,7 +907,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
                             for (UUID backupId : backups) {
                                 ClusterNode backup = cctx.discovery().node(backupId);
 
-                                if (backup != null && WAIT_REMOTE_TXS_SINCE.compareTo(backup.version()) <= 0) {
+                                if (backup != null) {
                                     if (backup.isLocal()) {
                                         IgniteInternalFuture<?> fut = cctx.tm().remoteTxFinishFuture(tx.nearXidVersion());
 
@@ -1077,7 +1035,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
         /**
          * @param nodes Backup nodes.
          */
-        public CheckRemoteTxMiniFuture(Set<UUID> nodes) {
+        CheckRemoteTxMiniFuture(Set<UUID> nodes) {
             this.nodes = nodes;
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d39050c7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
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 90a68ad..1b135d7 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
@@ -82,7 +82,6 @@ import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYS
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.UTILITY_CACHE_POOL;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM;
 import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isNearEnabled;
-import static org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishFuture.FINISH_NEAR_ONE_PHASE_SINCE;
 import static org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx.FinalizationStatus.USER_FINISH;
 import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
@@ -449,8 +448,7 @@ public class IgniteTxHandler {
 
             tx.transactionNodes(req.transactionNodes());
 
-            // Set near on originating node flag only if the sender node has new version.
-            if (req.near() && FINISH_NEAR_ONE_PHASE_SINCE.compareTo(nearNode.version()) <= 0)
+            if (req.near())
                 tx.nearOnOriginatingNode(true);
 
             if (req.onePhaseCommit()) {